You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/07/07 00:26:23 UTC

[GitHub] [pulsar] rdhabalia opened a new issue, #16424: PIP-136: Sync Pulsar policies across multiple clouds

rdhabalia opened a new issue, #16424:
URL: https://github.com/apache/pulsar/issues/16424

   # Sync Pulsar policies across multiple clouds
   
   Implementation : PR: https://github.com/apache/pulsar/pull/15223
   
   ## Motivation
   
   Apache Pulsar is a cloud-native, distributed messaging framework which natively provides geo-replication. Many organizations deploy pulsar instances on-prem and on multiple different cloud providers and at the same time they would like to enable replication between multiple clusters deployed in different cloud providers. Pulsar already provides various proxy options (Pulsar proxy/ enterprise proxy solutions on SNI) to fulfill security requirements when brokers are deployed on different security zones connected with each other.
   
   ![global metadata store -2](https://user-images.githubusercontent.com/2898254/177663404-b92d5acc-cf70-4e8d-96f0-5e7f3aa6bab6.png)
   
   However, sometimes it's not possible to share metadata-store (global zookeeper) between pulsar clusters deployed on separate cloud provider platforms, and synchronizing configuration metadata (policies) can be a critical path to share tenant/namespace/topic policies between clusters and administrate pulsar policies uniformly across all clusters. Therefore, we need a mechanism to sync configuration metadata between clusters deployed on the different cloud platforms.
   
   ## Goal
   
   ### Replicated metadata event topic
   All regions in a cluster that share the same metadata-store (eg: global zookeeper which persists policies) are already in sync but they are not in sync with regions which are in different clusters and not sharing the same metadata-store. We want to sync clusters that are not sharing the same config metadata store and in order to synchronize metadata store, we can pick one region from each cluster and set up a replicated topic across those regions where they can exchange change of metadata event and try to sync cluster with the change of events occurred at different clusters.
   
   This PIP will introduce metadata event-topic which will be replicated between isolated clusters that are not in sync and don’t share the same metadata store. In order to provide replication guarantee, each broker when it receives metadata update, first publishes metadata-change event to this replicated topic and then asynchronously applies the update to the metadata store by consuming metadata event.
   
   Below data structure shows the payload of change events published into the event topic. Each event contains metadata value along metadata path, source cluster name and last-updated time of the event. Source-cluster name and last-update time help destination clusters to handle stale or duplicate events. 
   
   ```
   public class MetadataEvent {
       private String path;
       private byte[] value;
       private Set<CreateOption> options;
       private Long expectedVersion;
       private long lastUpdatedTimestamp;
       private String sourceCluster;
       private NotificationType type;
   }
   ```
   
   ### Handling race condition
   Users can update the same policy with different values concurrently in different regions. Every region will eventually receive the updates from other remote regions where policy has been modified and Pulsar has to handle this scenario by merging (or selecting distinct value) the concurrent updates in a consistent manner across all regions. Therefore, each update contains modified-time and the name of source-region which has updated the value. Pulsar region compares local update and remote update based on latest modified timestamp and lexicographical ordering of source-region name and determines a final selected value deterministically across all regions and eventually all regions will have one distinct consistent value for the concurrently modified policy in the metadata store.
   
   For example in the below diagram, Region-A and Region-B received an update for policy P1 at the same time T1. Both regions exchange the local event update with each other and both the regions have to pick only one distinct event from both the updates so, both the regions will have a consistent same update in the metadata store. First, each region compares events based on event updated timestamp and then based on the lexicographic ordering of source-region name. In this example, modified timestamp T1 is the same for both the events so, the next Pulsar selects event with source region-name A over source-region B based on lexicographic sorting on source region name. Therefore, both regions will eventually update metadata with a distinct event that occurred at region-A.
   
   ![Race-condition](https://user-images.githubusercontent.com/2898254/153100353-4f966480-cfc4-422d-94ee-e714c62bd69b.jpeg)
   
   
   ## Implementation
   
   ### Event publisher and handler
   Every broker receives a metadata update publishes event message to the persistent topic (metadata-event-topic) which replicates to other clusters. Every region consumes events from this topic and applies changes to the metadata store accordingly. This PIP introduces MetadataEventSynchronizer to publish metadata events and consume events from the topic and handles updates in the metadata store.
   MetadataEventSynchronizer creates a failover consumer on the metadata-event-topic so, that only one of the brokers’ synchronizer can consume and handle the event update.
   
   **MetadataEventSynchronizer.java**
   
   ```
   **
    * Metadata synchronizer to notify and synchronize metadata change events.
    */
   public interface MetadataEventSynchronizer {
   
       /**
        * Notify metadata change event.
        * @param event
        *            metadata change event.
        * @return
        */
       CompletableFuture<Void> notify(MetadataEvent event);
   
       /**
        * Register notification listener to sync metadata event in local cluster.
        * @param event
        */
       void registerSyncListener(Function<MetadataEvent, CompletableFuture<Void>> event);
   
       /**
        * Name of current cluster served by the Synchronizer.
        * @return clusterName
        */
       String getClusterName();
   
       /**
        * close synchronizer resources.
        */
       void close();
   }
   ```
   
   ### Broker changes
   
   #### Configuration
   
   ```
   # topic name to share metadata changes from local metadata store
   private String metadataSyncEventTopic;
   
   # topic name to share metadata changes from configuration metadata store
   private long configurationMetadataSyncEventTopic;
   ```
   
   
   #### MetadataEventSynchronizer implementation
   `PulsarMetadataEventSynchronizer` which implements MetadataEventSynchronizer and handles metadata events’ notification and processing.
   
   ### Event topic consumer and publisher
   User can enable this feature by configuring metadataSyncEventTopic / configurationMetadataSyncEventTopic into broker and broker initializes MetadataEventSynchronizer component which creates failover consumer to listen and handle metadata’s change events. It also enables broker to publish metadata changes into event topic.
   
   PIP: 
   https://github.com/apache/pulsar/pull/15223
   
   ### Rejected alternative
   1. https://github.com/apache/pulsar/issues/13728
   2. Use System-topic:
   Use System-topic to synchronize metadata across the cluster. It might not be the correct choice to utilize system-topic to handle metadata-store transportation. Because system topic helps broker to persist topic policies in that local cluster whereas Metadata-event synchronizer helps broker to copy metadata-store across two independent clusters which don't share metadata-store/global-zookeeper. Users will also not be able to use system-topic  for metadata sync due to the below reasons:
   
   1. storage and reliability: Not every user prefers or uses the system topic for the metadata storage due to multiple reasons such as legacy-system, higher reliability on metadata-store compared to system-topic stored in bookies.
   2. Schema compatibility; System topic right now supports only topic level policies with a specific schema whereas the metadata change event requires a different schema for the metadata-store update.
   3. Merging and handling capabilities: Metadata change event not only requires different schema but also requires special handling for create/update and merging capabilities. It will require unnecessary enhancement on system-topic to support merging capabilities.
   4. Compaction requirement: system topic also requires compaction which all systems don't enable because compaction comes with an extra server-side cost which is very expensive for large scale and multi-tenant systems,
   
   However, system-topic can work with metadata-synchronizer. System topic persists topic policies. Broker reads this compacted system topic to retrieve topic policies and applies them to the loaded topic. The broker can replicate metadata-store data to another destination broker that is part of a separate cluster using a metadata-synchronizer, and the destination broker can later persist policies in the local cluster by publishing them to system-topic.
   
   
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] github-actions[bot] commented on issue #16424: PIP-136: Sync Pulsar policies across multiple clouds

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on issue #16424:
URL: https://github.com/apache/pulsar/issues/16424#issuecomment-1212664004

   The issue had no activity for 30 days, mark with Stale label.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org