Hi Enrico, Any further suggestion on this PIP? If not, I would like to raise a revote on this in a few days.
Thanks, Haiting On 2022/07/07 11:30:59 Haiting Jiang wrote: > Hi Enrico, > > Thanks for your feedback. > > On 2022/07/05 08:03:43 Enrico Olivelli wrote: > > I have a couple of additional questions. > > > > 1. Security > > What about security permissions about the shadow topic ? > > We are reading from another topic. > > I think we must clarify the decisions in the PIP > > As shadow topic is usually in another namespace, it would have its own > independent permission settings, and we can configure different permissions > for source topic and shadow topic. So there would be no guarantee that you are > allowed to consume shadow topic if you have permission to consume source > topic. > > On the other hand, we uses topic policy to store shadow topic settings, so a > new policy permission item needs be added as PolicyName.SHADOW_TOPIC, and user > must have PolicyOperation.WRITE to this policy to create/delete shadow topics. > > > > > 2. Truncation and deletion > > What happens when you truncate or delete the source topic ? > > please add a paragraph on the PIP > > > > 1. Truncation, from command `bin/pulsar-admin topics truncate source-topic`. > For source topic truncation, nothing changes. It still move all cursors to the > end of the topic and delete all inactive ledgers. > As shadow topic will watch `ManagedLedgerInfo` in metadata store, once it > knows ledgers deleted, all cursors will skip all deleted ledgers. > > 2. Deletion, from command `bin/pulsar-admin topics delete source-topic`. > Like geo-replication, topic deletion is forbidden if topic have shadow > replicators, users have to delete shadow topics first. Here is the new admin > API for managing shadow topics with source topic in > `org.apache.pulsar.client.admin.Topics` : > ``` > void createShadowTopic(String sourceTopicName, String shadowTopicName); > void deleteShadowTopic(String sourceTopicName, String shadowTopicName); > List<String> admin.topics().getShadowTopics(String sourceTopicName); > > //And their async version methods. > ``` > And this requires new REST interfaces in admin server, where > ``` > PATH = "/{tenant}/{namespace}/{topic}/shadowTopics"; > METHOD = POST/DELETE/GET; > ``` > > > 3. Offloaders > > We are talking about BK metadata, how do Shadow Topics work with > > Offloaded ledgers ? > > Please clarify in the PIP > > Offloading a ledger is a kind of writing operation to topic's metadata, so > shadow topic can't offload ledgers to other long term storage. However, for > ledgers thats are already offloaded by source topic, it's expected to support > reading from offload ledgers in shadow topic, just like read from source > topic. > > The implementation depends on shadow topic watching `ManagedLedgerInfo` in > metadata store, and if LedgerInfo.offloadContext is updated by source topic > offloader, shadow topic can get fully information to get a readHandle from > ledgerOffload. And of course, the pre-condition is the shadow topic must have > the same offload driver settings. > > > > > 4. Changes in the number of partitions > > the PIP says that the number of partitions must match the source topic. > > Are we preventing changes to the number of partitions in the source topic ? > > > > No, the updates on partition number will be synced to the shadow topic. > A source topic or partition will be responsible for the creation and deletion > of its corresponding shadow topic partitions. > > > 5. Topic stats > > We should add information on the source topic and on the shadow topic. > > Please clarify or draft your intentions in the PIP > > > > For topic stats on source topic, as shadow replicator will reuse most of > current > PersistentReplicator, the ReplicatorStatsImpl also can be applied to shadow > replicators. > And we need to add a new field in `TopicStatsImpl` like geo-replication: > ``` > Map<String /*shadow topic name*/, ReplicatorStatsImpl> shadowReplication; > ``` > > As for topic stats on shadow topic, previous `TopicStatsImpl` still applies. > And I don't see any other stats need to be added at this point. > > > > 6. GeoReplication > > I guess that GeoReplication will not be possible for shadow topics. > > Please clarify on the PIP > > > > Yes, this is decided by the nature of shadow topic that it don't have the > write access to BK. > And I don't see the necessary of supporting GeoReplication for shadow topics. > We can make source topic geo-replicated and create the same shadow topic in > each clusters. > > > I believe that this feature is very powerful, but we must design it > > carefully and discuss > > about all the edge cases. Otherwise we will end up in something that > > is half-baked > > and we will have to resolve edge cases while developing or after going > > to production. > > > > Every feature must be fully integrated with the rest of Pulsar > > > > Enrico > > > > Il giorno mer 29 giu 2022 alle ore 08:40 Haiting Jiang > > <jianghait...@apache.org> ha scritto: > > > > > > Hi Penghui > > > > > > On 2022/06/29 04:07:35 PengHui Li wrote: > > > > Hi Haiting, > > > > > > > > Thanks for the explanation. I'm clear for now. > > > > > > > > Pulsar functions also can do such things by connecting data from one > > > > topic > > > > to another topic. > > > > But the difference is this proposal only copies the data to the cache of > > > > another topic, and the data not > > > > in the cache is also available by reading from ledgers. > > > > > > > > And this approach also follows benefits compared with replicating data > > > > to > > > > multiple "real" topics. > > > > > > > > - reuse the topic metadata > > > > - the same message ID which easy for troubleshooting > > > > > > > > Just one question > > > > > > > > >>>>>>> > > > > ``` > > > > message CommandSend { // ... // message id for shadow topic optional > > > > MessageIdData shadow_message_id = 9; } > > > > ``` > > > > > > > > Can we get the message ID from the replicated data to avoid introducing > > > > a > > > > new command? > > > > Or use a marker message to avoid broker-to-broker directly protobuf > > > > command > > > > interaction. > > > > > > > Sorry for not wrote it clearly. CommandSend is not a new command. It's > > > exactly the main > > > command producer used to send message to broker. The only change is add a > > > new field in it. > > > The whole command proto would be like this: > > > ``` > > > message CommandSend { > > > required uint64 producer_id = 1; > > > required uint64 sequence_id = 2; > > > optional int32 num_messages = 3 [default = 1]; > > > optional uint64 txnid_least_bits = 4 [default = 0]; > > > optional uint64 txnid_most_bits = 5 [default = 0]; > > > > > > /// Add highest sequence id to support batch message with external > > > sequence id > > > optional uint64 highest_sequence_id = 6 [default = 0]; > > > optional bool is_chunk =7 [default = false]; > > > > > > // Specify if the message being published is a Pulsar marker or not > > > optional bool marker = 8 [default = false]; > > > > > > // message id for shadow topic > > > optional MessageIdData shadow_message_id = 9; > > > } > > > ``` > > > So there won't be any broker-to-broker directly protobuf command > > > interactions. > > > > > > Thanks, > > > Haiting > > > > > > > Thanks, > > > > Penghui > > > > > > > > On Wed, Jun 29, 2022 at 10:31 AM Haiting Jiang <jianghait...@apache.org> > > > > wrote: > > > > > > > > > Hi Penghui & Asaf: > > > > > > > > > > Please allow me to provide some more detailes about **metadata** > > > > > synchronization > > > > > between source topic and shadow topic. > > > > > > > > > > 1.When shadow topic initializes, it will read from metadata store path > > > > > "/managed-ledgers/{source_topic_ledger_name}", which contains all the > > > > > managed ledger info. We don't > > > > > need to read the ledger information from source topic broker. > > > > > > > > > > 2. When shadow topic received new message from replicator, if the > > > > > ledger > > > > > id of the message > > > > > is the same as the last ledger, it just updates the LAC. If not, it > > > > > will > > > > > update ledger list from metadata, > > > > > and then open the new ledger handle and update the LAC. > > > > > > > > > > As for the copy itself and add shadow message id in CommandSend, it > > > > > mostly > > > > > serves the purpose > > > > > of filling the EntryCache. > > > > > > > > > > Thanks, > > > > > Haiting > > > > > > > > > > On 2022/06/23 02:08:46 PengHui Li wrote: > > > > > > > One question comes to mind here: Why not simply read the ledger > > > > > information > > > > > > from original topic, without copy? > > > > > > > > > > > > I think this is a good idea. > > > > > > > > > > > > Penghui > > > > > > On Jun 22, 2022, 23:57 +0800, dev@pulsar.apache.org, wrote: > > > > > > > > > > > > > > One question comes to mind here: Why not simply read the ledger > > > > > information > > > > > > > from original topic, without copy? > > > > > > > > > > > > > > > > > > > BR, > Haiting >