Optimizing Airflow DAGs with Custom Macros: A Solution for Production Challenges

Anirban Saha
14 min readNov 24, 2023

--

Airflow users often employ macros to enhance their DAGs by dynamically passing data during runtime. There are numerous detailed blog posts available on the subject, with contributions from experts like Marc Lamberti.

In this article, I aim to delve into a recent challenge we encountered with our production DAGs and invite valuable feedback from our readers to refine our solution further.

The focus of this exploration centers on the utilization of custom macros to create functions that seamlessly integrate into your DAG definitions, eliminating the need for repeated function execution every time the DAG undergoes a refresh.

Setting Up The Context

Sometimes we might have a DAG with a PythonOperator, where the op_kwargs are obtained by calling some custom function (get_bucket_name in the example below):

my_task = PythonOperator(
task_id='my_task',
python_callable=my_function,
op_kwargs={
'bucket_name': get_bucket_name('model-blob')
},
)

Now, if this is a part of the DAG definition itself, this function will keep executing everytime the DAG refreshes.

from airflow import DAG
from airflow.operators.python_operator import PythonOperator
from datetime import datetime

def get_bucket_name(which_bucket):
if which_bucket == 'model-blob':
return 'models-gcs'
else:
return 'xyz'

default_args = {
'owner': 'airflow',
'start_date': datetime(2022, 1, 1),
}

with DAG('my_dag', default_args=default_args, schedule_interval='@daily') as dag:
my_task = PythonOperator(
task_id='my_task',
python_callable=my_function,
op_kwargs={
'bucket_name': get_bucket_name('model-blob'),
},
)

Setting up the Environment

To assess how a function behaves upon each DAG refresh, we’ve set up a Flask App that logs API calls and returns the count as a string. This string serves as an argument for the function invoked by a PythonOperator in our DAG.

For implementation, we’ll make slight adjustments to the docker-compose file provided in the Apache Documentation, adding an extra container to host the Flask application.

If you’re new to running Airflow with Docker, refer to the official documentation here for a comprehensive guide.

All of the code is available at my Github Profile — here.

Unoptimized DAG — Identifying Issues

I have a Flask App serving a simple app that logs the number of API calls made to it:

from flask import Flask

app = Flask(__name__)

call_count = 0

@app.route('/')
def count_calls():
global call_count
call_count += 1
return f'Total calls: {call_count}'

if __name__ == '__main__':
app.run(host='0.0.0.0', port=5001)

The Dockerfile for this app is simple:

# Use the official Python image as the base image
FROM python:3.8-slim

# Set the working directory in the container
WORKDIR /app

# Copy the current directory contents into the container at /app
COPY . /app

# Install any needed packages specified in requirements.txt
RUN pip install -r requirements.txt

# Expose the port on which the Flask app will run
EXPOSE 5001

# Define the command to run your application
CMD ["python", "app.py"]

The requirements just has one library:

flask==3.0.0

So, if we run this with the container port 5001 mapped to host port 5001 (or any available port), using

docker build --pull --rm -f "Dockerfile" -t airflowmacrosdemo:latest "." 
docker run --name flask-app -p 5001:5001 airflowmacrosdemo:latest

We will see something like:

Ping Flask Endpoint to see Count — 1

If we refresh and ping again, we will see

Ping Flask Endpoint to see Count — 2

Now, we will use this container as a part of our Airflow setup making minor changes to the Airflow docker-compose file found in the official documentation.

# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
#

# Basic Airflow cluster configuration for CeleryExecutor with Redis and PostgreSQL.
#
# WARNING: This configuration is for local development. Do not use it in a production deployment.
#
# This configuration supports basic configuration using environment variables or an .env file
# The following variables are supported:
#
# AIRFLOW_IMAGE_NAME - Docker image name used to run Airflow.
# Default: apache/airflow:2.7.2
# AIRFLOW_UID - User ID in Airflow containers
# Default: 50000
# AIRFLOW_PROJ_DIR - Base path to which all the files will be volumed.
# Default: .
# Those configurations are useful mostly in case of standalone testing/running Airflow in test/try-out mode
#
# _AIRFLOW_WWW_USER_USERNAME - Username for the administrator account (if requested).
# Default: airflow
# _AIRFLOW_WWW_USER_PASSWORD - Password for the administrator account (if requested).
# Default: airflow
# _PIP_ADDITIONAL_REQUIREMENTS - Additional PIP requirements to add when starting all containers.
# Use this option ONLY for quick checks. Installing requirements at container
# startup is done EVERY TIME the service is started.
# A better way is to build a custom image or extend the official image
# as described in https://airflow.apache.org/docs/docker-stack/build.html.
# Default: ''
#
# Feel free to modify this file to suit your needs.
---
version: '3.8'
x-airflow-common:
&airflow-common
# In order to add custom dependencies or upgrade provider packages you can use your extended image.
# Comment the image line, place your Dockerfile in the directory where you placed the docker-compose.yaml
# and uncomment the "build" line below, Then run `docker-compose build` to build the images.
image: ${AIRFLOW_IMAGE_NAME:-apache/airflow:2.7.2}
# build: .
environment:
&airflow-common-env
AIRFLOW__CORE__EXECUTOR: CeleryExecutor
AIRFLOW__DATABASE__SQL_ALCHEMY_CONN: postgresql+psycopg2://airflow:airflow@postgres/airflow
# For backward compatibility, with Airflow <2.3
AIRFLOW__CORE__SQL_ALCHEMY_CONN: postgresql+psycopg2://airflow:airflow@postgres/airflow
AIRFLOW__CELERY__RESULT_BACKEND: db+postgresql://airflow:airflow@postgres/airflow
AIRFLOW__CELERY__BROKER_URL: redis://:@redis:6379/0
AIRFLOW__CORE__FERNET_KEY: ''
AIRFLOW__CORE__DAGS_ARE_PAUSED_AT_CREATION: 'true'
AIRFLOW__CORE__LOAD_EXAMPLES: 'false'
AIRFLOW__API__AUTH_BACKENDS: 'airflow.api.auth.backend.basic_auth,airflow.api.auth.backend.session'
# yamllint disable rule:line-length
# Use simple http server on scheduler for health checks
# See https://airflow.apache.org/docs/apache-airflow/stable/administration-and-deployment/logging-monitoring/check-health.html#scheduler-health-check-server
# yamllint enable rule:line-length
AIRFLOW__SCHEDULER__ENABLE_HEALTH_CHECK: 'true'
# WARNING: Use _PIP_ADDITIONAL_REQUIREMENTS option ONLY for a quick checks
# for other purpose (development, test and especially production usage) build/extend Airflow image.
_PIP_ADDITIONAL_REQUIREMENTS: ${_PIP_ADDITIONAL_REQUIREMENTS:-}
volumes:
- ${AIRFLOW_PROJ_DIR:-.}/dags:/opt/airflow/dags
- ${AIRFLOW_PROJ_DIR:-.}/logs:/opt/airflow/logs
- ${AIRFLOW_PROJ_DIR:-.}/config:/opt/airflow/config
- ${AIRFLOW_PROJ_DIR:-.}/plugins:/opt/airflow/plugins
user: "${AIRFLOW_UID:-50000}:0"
depends_on:
&airflow-common-depends-on
redis:
condition: service_healthy
postgres:
condition: service_healthy

services:
postgres:
image: postgres:13
environment:
POSTGRES_USER: airflow
POSTGRES_PASSWORD: airflow
POSTGRES_DB: airflow
volumes:
- postgres-db-volume:/var/lib/postgresql/data
healthcheck:
test: ["CMD", "pg_isready", "-U", "airflow"]
interval: 10s
retries: 5
start_period: 5s
restart: always

flask-app:
build:
context: . # Path to your Flask app directory
ports:
- "5001:5001"

redis:
image: redis:latest
expose:
- 6379
healthcheck:
test: ["CMD", "redis-cli", "ping"]
interval: 10s
timeout: 30s
retries: 50
start_period: 30s
restart: always

airflow-webserver:
<<: *airflow-common
command: webserver
ports:
- "8080:8080"
healthcheck:
test: ["CMD", "curl", "--fail", "http://localhost:8080/health"]
interval: 30s
timeout: 10s
retries: 5
start_period: 30s
restart: always
depends_on:
<<: *airflow-common-depends-on
airflow-init:
condition: service_completed_successfully

airflow-scheduler:
<<: *airflow-common
command: scheduler
healthcheck:
test: ["CMD", "curl", "--fail", "http://localhost:8974/health"]
interval: 30s
timeout: 10s
retries: 5
start_period: 30s
restart: always
depends_on:
<<: *airflow-common-depends-on
airflow-init:
condition: service_completed_successfully

airflow-worker:
<<: *airflow-common
command: celery worker
healthcheck:
# yamllint disable rule:line-length
test:
- "CMD-SHELL"
- 'celery --app airflow.providers.celery.executors.celery_executor.app inspect ping -d "celery@$${HOSTNAME}" || celery --app airflow.executors.celery_executor.app inspect ping -d "celery@$${HOSTNAME}"'
interval: 30s
timeout: 10s
retries: 5
start_period: 30s
environment:
<<: *airflow-common-env
# Required to handle warm shutdown of the celery workers properly
# See https://airflow.apache.org/docs/docker-stack/entrypoint.html#signal-propagation
DUMB_INIT_SETSID: "0"
restart: always
depends_on:
<<: *airflow-common-depends-on
airflow-init:
condition: service_completed_successfully

airflow-triggerer:
<<: *airflow-common
command: triggerer
healthcheck:
test: ["CMD-SHELL", 'airflow jobs check --job-type TriggererJob --hostname "$${HOSTNAME}"']
interval: 30s
timeout: 10s
retries: 5
start_period: 30s
restart: always
depends_on:
<<: *airflow-common-depends-on
airflow-init:
condition: service_completed_successfully

airflow-init:
<<: *airflow-common
entrypoint: /bin/bash
# yamllint disable rule:line-length
command:
- -c
- |
function ver() {
printf "%04d%04d%04d%04d" $${1//./ }
}
airflow_version=$$(AIRFLOW__LOGGING__LOGGING_LEVEL=INFO && gosu airflow airflow version)
airflow_version_comparable=$$(ver $${airflow_version})
min_airflow_version=2.2.0
min_airflow_version_comparable=$$(ver $${min_airflow_version})
if (( airflow_version_comparable < min_airflow_version_comparable )); then
echo
echo -e "\033[1;31mERROR!!!: Too old Airflow version $${airflow_version}!\e[0m"
echo "The minimum Airflow version supported: $${min_airflow_version}. Only use this or higher!"
echo
exit 1
fi
if [[ -z "${AIRFLOW_UID}" ]]; then
echo
echo -e "\033[1;33mWARNING!!!: AIRFLOW_UID not set!\e[0m"
echo "If you are on Linux, you SHOULD follow the instructions below to set "
echo "AIRFLOW_UID environment variable, otherwise files will be owned by root."
echo "For other operating systems you can get rid of the warning with manually created .env file:"
echo " See: https://airflow.apache.org/docs/apache-airflow/stable/howto/docker-compose/index.html#setting-the-right-airflow-user"
echo
fi
one_meg=1048576
mem_available=$$(($$(getconf _PHYS_PAGES) * $$(getconf PAGE_SIZE) / one_meg))
cpus_available=$$(grep -cE 'cpu[0-9]+' /proc/stat)
disk_available=$$(df / | tail -1 | awk '{print $$4}')
warning_resources="false"
if (( mem_available < 4000 )) ; then
echo
echo -e "\033[1;33mWARNING!!!: Not enough memory available for Docker.\e[0m"
echo "At least 4GB of memory required. You have $$(numfmt --to iec $$((mem_available * one_meg)))"
echo
warning_resources="true"
fi
if (( cpus_available < 2 )); then
echo
echo -e "\033[1;33mWARNING!!!: Not enough CPUS available for Docker.\e[0m"
echo "At least 2 CPUs recommended. You have $${cpus_available}"
echo
warning_resources="true"
fi
if (( disk_available < one_meg * 10 )); then
echo
echo -e "\033[1;33mWARNING!!!: Not enough Disk space available for Docker.\e[0m"
echo "At least 10 GBs recommended. You have $$(numfmt --to iec $$((disk_available * 1024 )))"
echo
warning_resources="true"
fi
if [[ $${warning_resources} == "true" ]]; then
echo
echo -e "\033[1;33mWARNING!!!: You have not enough resources to run Airflow (see above)!\e[0m"
echo "Please follow the instructions to increase amount of resources available:"
echo " https://airflow.apache.org/docs/apache-airflow/stable/howto/docker-compose/index.html#before-you-begin"
echo
fi
mkdir -p /sources/logs /sources/dags /sources/plugins
chown -R "${AIRFLOW_UID}:0" /sources/{logs,dags,plugins}
exec /entrypoint airflow version
# yamllint enable rule:line-length
environment:
<<: *airflow-common-env
_AIRFLOW_DB_MIGRATE: 'true'
_AIRFLOW_WWW_USER_CREATE: 'true'
_AIRFLOW_WWW_USER_USERNAME: ${_AIRFLOW_WWW_USER_USERNAME:-airflow}
_AIRFLOW_WWW_USER_PASSWORD: ${_AIRFLOW_WWW_USER_PASSWORD:-airflow}
_PIP_ADDITIONAL_REQUIREMENTS: ''
user: "0:0"
volumes:
- ${AIRFLOW_PROJ_DIR:-.}:/sources

airflow-cli:
<<: *airflow-common
profiles:
- debug
environment:
<<: *airflow-common-env
CONNECTION_CHECK_MAX_COUNT: "0"
# Workaround for entrypoint issue. See: https://github.com/apache/airflow/issues/16252
command:
- bash
- -c
- airflow

# You can enable flower by adding "--profile flower" option e.g. docker-compose --profile flower up
# or by explicitly targeted on the command line e.g. docker-compose up flower.
# See: https://docs.docker.com/compose/profiles/
flower:
<<: *airflow-common
command: celery flower
profiles:
- flower
ports:
- "5555:5555"
healthcheck:
test: ["CMD", "curl", "--fail", "http://localhost:5555/"]
interval: 30s
timeout: 10s
retries: 5
start_period: 30s
restart: always
depends_on:
<<: *airflow-common-depends-on
airflow-init:
condition: service_completed_successfully

volumes:
postgres-db-volume:

The only change we have done is

flask-app:
build:
context: . # Path to your Flask app directory
ports:
- "5001:5001"

Once we do docker-compose up , it will set up a local airflow environment for us.

Our next step is to examine an unoptimized DAG that repeatedly calls the Flask endpoint to update the op_kwargs value for the PythonOperator. Over time, we'll observe how this process accumulates the number of API calls made to our endpoint.

from airflow import DAG
from airflow.operators.python_operator import PythonOperator
from datetime import datetime
import requests

default_args = {
'owner': 'Anirban Saha',
'start_date': datetime(2023, 10, 15),
'retries': 1,
}
FLASK_ENDPOINT_URL="http://flask-app:5001"
dag = DAG(
'flask_call_dag',
default_args=default_args,
schedule_interval=None,
catchup=False,
)
def get_flask_response():
try:
response = requests.get(FLASK_ENDPOINT_URL)
if response.status_code == 200:
print("Response - ", response.text)
return response.text
else:
return f"Request failed with status code {response.status_code}"
except requests.exceptions.RequestException as e:
return f"An error occurred: {e}"

def print_flask_response(str_= None):
print(f"The Function returned the following output: {str_}")

print_response_task = PythonOperator(
task_id='print_response_task',
python_callable=print_flask_response,
op_kwargs={"str_": get_flask_response()},
dag=dag,
)

print_response_task

We can see, the current number of API calls is 3:

Checking API calls — 3

We will wait for sometime, and check again to see, total number of API calls increases to 62:

Checking API calls — 62

The reason behind this behavior is that, with every DAG refresh, the get_flask_response function is triggered, resulting in an API call.

In our production code, we encountered a similar scenario where we used a function like this to populate the job parameter of the DataprocSubmitJobOperator. When such functions involve resource-intensive API calls or interactions with services like Google Bigquery, it not only hampers DAG rendering speed but also leads to unwarranted expenses.

In our quest for solutions, we will explore two approaches: harnessing Airflow’s XCom and crafting custom macros. But before delving into these alternatives, we’ll temporarily comment out our existing DAG code to halt these redundant API calls, and reset the count. It’s essential to remember that these API calls occur not only when our task is executed but also with each DAG refresh.

Using Airflow XCom

from airflow import DAG
from airflow.operators.python_operator import PythonOperator
from datetime import datetime
import requests

default_args = {
'owner': 'Anirban Saha',
'start_date': datetime(2023, 10, 15),
'retries': 1,
}
FLASK_ENDPOINT_URL="http://flask-app:5001"
dag = DAG(
'xcom_dag',
default_args=default_args,
schedule_interval=None,
catchup=False,
)
def get_flask_response(**kwargs):
try:
response = requests.get(FLASK_ENDPOINT_URL)
if response.status_code == 200:
result = response.text
print("Response - ", result)
kwargs['ti'].xcom_push(key='flask_response', value=result) # Push the result to XCom
return result
else:
return f"Request failed with status code {response.status_code}"
except requests.exceptions.RequestException as e:
return f"An error occurred: {e}"

def print_flask_response(**kwargs):
ti = kwargs['ti']
response_text = ti.xcom_pull(task_ids='get_flask_response_task', key='flask_response')
print(f"The Function returned the following output: {response_text}")

get_flask_response_task = PythonOperator(
task_id='get_flask_response_task',
python_callable=get_flask_response,
provide_context=True,
dag=dag,
)

print_response_task = PythonOperator(
task_id='print_response_task',
python_callable=print_flask_response,
provide_context=True,
dag=dag,
)

get_flask_response_task >> print_response_task # Set the task dependency

The official Airflow Documentation describes XComs as:

XComs (short for “cross-communications”) are a mechanism that let Tasks talk to each other, as by default Tasks are entirely isolated and may be running on entirely different machines.
An XCom is identified by a key (essentially its name), as well as the task_id and dag_id it came from. They can have any (serializable) value, but they are only designed for small amounts of data; do not use them to pass around large values, like dataframes.
XComs are explicitly “pushed” and “pulled” to/from their storage using the xcom_push and xcom_pull methods on Task Instances.

In pursuit of optimizing our workflow by minimizing the frequency of continuous function calls, we’ve improved our approach by moving the function that fills in the details to a separate step. Then, we execute this step as its own task, sending the results to Airflow’s XCom feature. In our primary function, we retrieve these values from XCom, thereby achieving a more streamlined and modularized architecture.

Now our DAG looks like this:

XCom DAG Structure

Now our task get_flask_response_task will retrieve the value from the API endpoint, and it gets pushed to the XCom for the task:

XCom for `get_flask_response_task`

This value is then pulled from XCom in our print_response_task .

Log for`print_response_task`

Subsequent to this modification, refreshing our API endpoint at intervals no longer reveals an influx of numerous calls, as the DAG has been optimized to refrain from incessantly invoking the endpoint with each refresh.

Checking API calls — 4

But, what if we have tons of tasks in Airflow and we need to fill in details for each one? Imagine we’re making hundreds of tasks on the fly, and for each of them, we want to add 2–3 details. Making hundreds of XCom tasks might make our DAGs look a bit messy.

This introduces an unnecessary layer of complexity to our DAGs, impacting both readability and the ease of debugging.

In such cases, we’ll dig into using custom macros in Airflow.

Custom Macros in Airflow

Macros are a way to expose objects to your templates and live under the macros namespace in your templates
— Airflow Documentation

By leveraging Airflow Macros, we can eliminate the need for constant calls to our API endpoint. Now, the API call only happens when our task is executed, triggered by the rendering of the template. Opting for Airflow Macros instead of XCom not only streamlines the DAG but also contributes to a simpler and more easily readable workflow.

Templates and Macros in Apache Airflow allow passing data to your DAGs at runtime.
— Marc Lameberti

I’ve crafted a function, referred to as the macro function, that communicates with the API endpoint to fetch the string, subsequently passed as an argument to our main function. To demonstrate the utilization of arguments within our macro function, I’ve included a dummy argument for clarity.

def get_flask_response_wth_arguments(op_str=None):
try:
response = requests.get(FLASK_ENDPOINT_URL)
if response.status_code == 200:
result = response.text
print(f"Response - {op_str} - ", result)
return result
else:
return f"Request failed with status code {response.status_code}"
except requests.exceptions.RequestException as e:
return f"An error occurred: {e}"

And our DAG code now looks like:

from airflow import DAG
from airflow.operators.python_operator import PythonOperator
from datetime import datetime
from macros import get_flask_response_wth_arguments

default_args = {
'owner': 'Anirban Saha',
'start_date': datetime(2023, 10, 15),
'retries': 1,
}
dag = DAG(
'macros__with_arguments_dag',
default_args=default_args,
schedule_interval=None,
user_defined_macros = {'get_flask_response_wth_arguments': get_flask_response_wth_arguments},
render_template_as_native_obj = True,
catchup=False,
)

def print_flask_response(str_):
print(f"The Function returned the following output: {str_}")

op_str_='abc'
print_response_task = PythonOperator(
task_id='print_response_task',
python_callable=print_flask_response,
op_kwargs = {"str_": f"{{{{ get_flask_response_wth_arguments(op_str='{op_str_}') }}}}"},
provide_context=True,
dag=dag,
)

print_response_task # Set the task dependency

Take note of a couple of key adjustments: specifically, the introduction of

user_defined_macros = {‘get_flask_response_wth_arguments’: get_flask_response_wth_arguments} 

signals to Airflow that the function employed in

op_kwargs = {“str_”: f”{{{{ get_flask_response_wth_arguments(op_str=’{op_str_}’) }}}}”}

is indeed a macro.

It’s crucial to observe the encapsulation of get_flask_response_wth_arguments within {{{{ }}}}in this Jinja template. This template comes to life only when the task is triggered, ensuring it doesn’t execute with every DAG refresh, contributing to a more efficient workflow.

Upon task execution, the corresponding DAG is rendered as follows:

Custom Macro Rendered
Airflow Template Rendering Steps

It’s important to recognize that not all parameters of Airflow Operators support template values. Specific parameters are designed to accommodate templatization. To determine if an operator allows templating for a particular parameter and to identify which parameter it is, it’s advisable to inspect the code associated with that operator.

Each operator comes equipped with two pivotal parameters for this purpose. The first is `template_fields`, delineating which parameters are eligible for templating. The second, `template_ext`, outlines the file extensions that can leverage template values. Delving into these aspects in the operator’s code offers insights into the templating capabilities and enhances our understanding of how to wield them effectively.

As an illustration, let’s take a look at the DataprocSubmitPySparkJobOperator here.

class DataprocSubmitPySparkJobOperator(DataprocJobBaseOperator):
"""Start a PySpark Job on a Cloud DataProc cluster.

.. seealso::
This operator is deprecated, please use
:class:`~airflow.providers.google.cloud.operators.dataproc.DataprocSubmitJobOperator`:

:param main: [Required] The Hadoop Compatible Filesystem (HCFS) URI of the main
Python file to use as the driver. Must be a .py file. (templated)
:param arguments: Arguments for the job. (templated)
:param archives: List of archived files that will be unpacked in the work
directory. Should be stored in Cloud Storage.
:param files: List of files to be copied to the working directory
:param pyfiles: List of Python files to pass to the PySpark framework.
Supported file types: .py, .egg, and .zip
"""

template_fields: Sequence[str] = (
"main",
"arguments",
"job_name",
"cluster_name",
"region",
"dataproc_jars",
"dataproc_properties",
"impersonation_chain",
)

Within this operator, you’ll observe that certain fields, namely main, arguments, job_name, cluster_name, region, dataproc_jars, dataproc_properties, impersonation_chain are open to templating. These fields provide the flexibility to inject dynamic values, enhancing the adaptability and robustness of your workflows.

In our production pipeline, we effectively leveraged the jobparameter of the DataprocSubmitJobOperator(link), resulting in a notable reduction in continuous calls to both Google BigQuery (known for its cost implications) and our internal Config Management System (which tends to be slow). This optimization significantly contributed to the efficiency and cost-effectiveness of our workflow.

If you have already made it this far, you might like my style of writing (pretty please!?). I write (or try to write) about Tech and my travels.

I’d love for you to follow me on my:

Instagram (for Travel content) — https://www.instagram.com/anirban.saha/
or LinkedIn (for tech content) —
https://www.linkedin.com/in/saha-anirban8/
Substack (Getting started) —
https://anirbans.substack.com/

--

--

Anirban Saha

Rediscovering self love. Love documenting my travel experiences and writing about the little tech I know!