Cluster migration or Blue-Green cluster deployment is one of the proven solutions to migrate live traffic from one cluster to another. One of the examples is applications running on Kubernetes sometimes require a Kubernetes cluster upgrade which can cause downtime for the entire application during a Kubernetes cluster upgrade. Blue-green deployment is an application release model that gradually transfers user traffic from a previous version of an app or microservice to a nearly identical new release—both of which are running in production.
The old version can be called the blue environment while the new version can be known as the green environment. Once production traffic is fully transferred from blue to green, blue can standby in case of rollback or be pulled from production and updated to become the template upon which the next update is made.
We need such capability in Apache pulsar to migrate live traffic from the blue cluster to the green cluster so, eventually, the entire traffic moves from the blue cluster to the green cluster without causing downtime for the topics.
This PIP adds support to migrate and redirect the blue cluster’s traffic to the green cluster. Therefore, the Broker will support admin-API using which admin-user can mark migrate cluster along with redirection URLs where traffic should be redirected. Broker persists migration state and new redirected cluster’s URL as part of cluster metadata.
Once the cluster is marked as migrating, the broker asynchronously marks each topic owned by that broker as migrated by calling the new managed-ledger API asyncMigrate()
. Once, the topic is marked as migrated, broker notifies all the producers and consumers (which have drained the backlog for their subscriptions) with a new client-protocol command called “Migrated-Topic” which has redirection URLs to the green cluster. Producers and consumers for those topics cache the redirection URLs and retry to connect to the broker with that URL which redirects them to the green cluster.
Broker can redirect only those consumers which have reached to end of a terminated topic or create a new subscription in the blue cluster. Therefore, the broker can determine the redirection of consumers in the consumer-creation phase and the pulsar client has to handle redirection after sending producer/consumer creation requests.
Broker will unsubscribe the subscription once that subscription reaches end of topic and broker will also not allow creation of any new producer or subscription for the topics. Therefore, eventually, all the topics in the blue cluster will not have a subscription or producer attached and eventually, those topics will be deleted by the garbage collector.
Broker marks the cluster state as migration-completed once all the topics are deleted and that cluster will not allow any new topic creation.
This PIP depends on the recently added broker's feature to terminate topic. In case, if any user decides to terminate the topic then it can be done using admin API. Once, broker receives request to terminate the topic, broker
Terminated
and doesn't allow any new writes on that managed-ledger.TopicTerminatedError
Migration of topic will have enhancement on top of topic-termination feature where migration process will
TopicTerminatedError
error to the producer, the broker sends migration-response to producer so, producer can handle migration-response and manage redirection to the new cluster. We will discuss this step in detail in next section.This PIP will add API to managed-ledger to change ManagedLedger state as migrated. This API will terminate the topic and persist the status of managed-ledger as migrated. Broker triggers migrate API of managed-ledger once cluster becomes the blue cluster and traffic should redirect to the green cluster.
ManagedLedger.java CompletableFuture<Position> asyncMigrate();
It will be persisted as a part of ManagedLedgerInfo::properties
so, broker can recover migrated topic/ledger upon broker restart.
Pulsar Admin Change
pulsar-admin clusters set-cluster-migrated \ –brokerServiceUrl <> \ –brokerServiceUrlTls <>
Pulsar stores each region’s url metadata into global zookeeper or configuration metadata store as a cluster metadata for that region where each cluster’s metadata stores that region’s broker service urls. During the migration of a given blue region to its appropriate green region, blue region needs to store the migration-state flag and then appropriate green region’s cluster metadata URL where blue region can redirect the traffic. Both blue and green regions can share the same configuration metadata store or global zookeeper and the migration state of a blue region is attached to that blue region only. Therefore, blue region must store migration state and redirection url metadata locally in a local metadata store as this metadata configuration is only applicable to the blue region and not to be shared with green region. Therefore, pulsar broker will store the migration policies for the blue cluster at local metadata-store at path: /admin/<cluster-name>/policies
and it will store below migration metadata info in that cluster policies.
Cluster-metadata change
ClusterPolicies.java boolean migrated; ClusterUrl migratedClusterUrl; class ClusterUrl { String brokerServiceUrl; String brokerServiceUrlTls; }
Sometimes, we would like to start migration namespace by namespace, and admin requires a namespace level control for the migration so, it can start and test migration setup first with specific namespaces. Therefore, Pulsar should also allow controlling migration at the namespace level and Pulsar should provide admin API to enable migration for specific namespace. Pulsar broker will persist migration enable flag into namespace local policies. Therefore, the Pulsar broker will provide API to update and retrieve the namespace migration state.
Namespaces.java
void updateMigrationState(String namespace, boolean migrated) throws PulsarAdminException;
It’s really important to monitor the migration process and progress during blue-green cluster migration. Therefore, Pulsar will have a migration flag into existing topic stats so, we can monitor the migration state for each topic.
ServiceConfiguration.java private long clusterMigrationCheckDurationSeconds = 0; //disable task with default value=0
PulsarApi.proto message CommandTopicMigrated { enum ResourceType { Producer = 0; Consumer = 1; } required uint64 resource_id = 1; required ResourceType resource_type = 2; optional string brokerServiceUrl = 2; optional string brokerServiceUrlTls = 3; }
Once, topic is marked as migrated, broker will start redirecting producers to new cluster to publish new messages but if consumers are not yet redirected then messages at green cluster can be lost. Therefore, we can apply default retention policy at Green cluster until blue cluster is migrated. So, Green cluster will retain the messages until blue cluster is completely migrated to green cluster but that can make green cluster store large amount of retention data which might go beyond storage capacity of green cluster. Therefore, another option is to allow blue broker to create necessary resources of a topic such as tenant, namespace and list of existing subscription in a green cluster before starting migration and green cluster will persist newly published data until all consumers successfully migrated to green cluster and start catching up with the published messages.
In ordered to achieve cluster migration, all clients need to be upgraded to the version which handles cluster migration redirection. So, Broker can‘t complete migration if clients are not upgrade to the supported version. Therefore, it’s really important to find out if any unsupported client is connected or don‘t allow any unsupported client to connect if we want to start the cluster migration process. Therefore, we will add a flag to allow the broker to reject all client connections which are initiated from unsupported clients. The broker will not allow connection from clients which don’t support cluster migration redirection handling if the clientMinVersionAllowed
broker flag is enabled.
Once topic is successfully migrated from blue cluster to green cluster, brokers in a blue cluster should unsubscribe all subscription and garbage collector should be able to delete that topic successfully. Once, all the topics are migrated successfully to the green cluster, blue cluster will not have any single topic exist into the cluster which shows that blue green cluster migration is successfully finished. However, Pulsar broker should also makes sure that topic must not be recreated after migrating to green cluster successfully.
A. Incoming replication messages from other region's replicator producers to Blue cluster This will not impact ordering messages coming from the other regions to blue/green cluster. After marking blue cluster, blue cluster will reject replication writes from remote regions and redirects remote producers to the Green cluster where new messages will be written. Consumers of Blue clusters will only be redirected to green once they received all messages from blue. So, migration gives an ordering guarantee for messages replicating from remote regions.
B. Outgoing replication messages from Blue cluster's replicator producers to other regions The broker can give an ordering guarantee in this case with the trade-off of topic unavailability until the blue cluster replicates all existing published messages in the blue cluster before the topic gets terminated.
TOPIC_UNAVAILABLE
message to producers/consumers so, they can keep retrying until replicators reach to end of topics.Once producers and consumers receive the Migrated-Topic command with a list of redirect URLs, they will cache those URLs and try to reconnect with a broker by using those URLs. The client will add handling of the CommandMigratedTopic
protocol.