mjsax commented on code in PR #13252: URL: https://github.com/apache/kafka/pull/13252#discussion_r1110445053
########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java: ########## @@ -0,0 +1,227 @@ +/* + * 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. + */ +package org.apache.kafka.streams.state.internals; + +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; + +import java.util.Objects; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.query.Position; +import org.apache.kafka.streams.query.PositionBound; +import org.apache.kafka.streams.query.Query; +import org.apache.kafka.streams.query.QueryConfig; +import org.apache.kafka.streams.query.QueryResult; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.VersionedBytesStore; +import org.apache.kafka.streams.state.VersionedKeyValueStore; +import org.apache.kafka.streams.state.VersionedRecord; + +/** + * A metered {@link VersionedKeyValueStore} wrapper that is used for recording operation + * metrics, and hence its inner {@link VersionedBytesStore} implementation does not need to provide + * its own metrics collecting functionality. The inner {@code VersionedBytesStore} of this class + * is a {@link KeyValueStore} of type <Bytes,byte[]>, so we use {@link Serde}s + * to convert from <K,ValueAndTimestamp<V>> to <Bytes,byte[]>. In particular, + * {@link NullableValueAndTimestampSerde} is used since putting a tombstone to a versioned key-value + * store requires putting a null value associated with a timestamp. + * + * @param <K> The key type + * @param <V> The (raw) value type + */ +public class MeteredVersionedKeyValueStore<K, V> + extends WrappedStateStore<VersionedBytesStore, K, V> + implements VersionedKeyValueStore<K, V> { + + private final MeteredVersionedKeyValueStoreInternal internal; + + MeteredVersionedKeyValueStore(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner); + internal = new MeteredVersionedKeyValueStoreInternal(inner, metricScope, time, keySerde, valueSerde); + } + + /** + * Private helper class which represents the functionality of a {@link VersionedKeyValueStore} + * as a {@link TimestampedKeyValueStore} so that the bulk of the metering logic may be + * inherited from {@link MeteredKeyValueStore}. As a result, the implementation of + * {@link MeteredVersionedKeyValueStore} is a simple wrapper to translate from this + * {@link TimestampedKeyValueStore} representation of a versioned key-value store into the + * {@link VersionedKeyValueStore} interface itself. Review Comment: I think this comments need some more information. What about this: ``` Conceptually, MeteredVersionedKeyValueStore should <code>extend</code> MeteredTimestampKeyValueStore, but due to type conflicts, we cannot do this. Thus, we use an <it>instance</it> of MeteredTimestampKeyValueStore to mimic inheritance instead. This instance of MeteredTimestampKeyValueStore wraps the inner VersionedKeyValueStore, and thus we need to do some type translation inside the internal wrapped below. It's not ideal in the sense that we need to translate between the APIs/types twice, but reusing code is still better than c&p it. Note that we overwrite `get()` but newly add `put(k, v, ts)` as `delete(k, ts)` to get the same API as defined on VersionedKeyValueStore. ``` ########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java: ########## @@ -0,0 +1,227 @@ +/* + * 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. + */ +package org.apache.kafka.streams.state.internals; + +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; + +import java.util.Objects; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.query.Position; +import org.apache.kafka.streams.query.PositionBound; +import org.apache.kafka.streams.query.Query; +import org.apache.kafka.streams.query.QueryConfig; +import org.apache.kafka.streams.query.QueryResult; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.VersionedBytesStore; +import org.apache.kafka.streams.state.VersionedKeyValueStore; +import org.apache.kafka.streams.state.VersionedRecord; + +/** + * A metered {@link VersionedKeyValueStore} wrapper that is used for recording operation + * metrics, and hence its inner {@link VersionedBytesStore} implementation does not need to provide + * its own metrics collecting functionality. The inner {@code VersionedBytesStore} of this class + * is a {@link KeyValueStore} of type <Bytes,byte[]>, so we use {@link Serde}s + * to convert from <K,ValueAndTimestamp<V>> to <Bytes,byte[]>. In particular, + * {@link NullableValueAndTimestampSerde} is used since putting a tombstone to a versioned key-value + * store requires putting a null value associated with a timestamp. + * + * @param <K> The key type + * @param <V> The (raw) value type + */ +public class MeteredVersionedKeyValueStore<K, V> + extends WrappedStateStore<VersionedBytesStore, K, V> + implements VersionedKeyValueStore<K, V> { + + private final MeteredVersionedKeyValueStoreInternal internal; + + MeteredVersionedKeyValueStore(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner); + internal = new MeteredVersionedKeyValueStoreInternal(inner, metricScope, time, keySerde, valueSerde); + } + + /** + * Private helper class which represents the functionality of a {@link VersionedKeyValueStore} + * as a {@link TimestampedKeyValueStore} so that the bulk of the metering logic may be + * inherited from {@link MeteredKeyValueStore}. As a result, the implementation of + * {@link MeteredVersionedKeyValueStore} is a simple wrapper to translate from this + * {@link TimestampedKeyValueStore} representation of a versioned key-value store into the + * {@link VersionedKeyValueStore} interface itself. + */ + private class MeteredVersionedKeyValueStoreInternal + extends MeteredKeyValueStore<K, ValueAndTimestamp<V>> + implements TimestampedKeyValueStore<K, V> { + + private final VersionedBytesStore inner; + + MeteredVersionedKeyValueStoreInternal(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner, metricScope, time, keySerde, valueSerde); + this.inner = inner; + } + + @Override + public void put(final K key, final ValueAndTimestamp<V> value) { + super.put( Review Comment: Thinking about this once more, it seems not ideal to do it this way? We get `ValueAndTimestamp` and call `super.put()`, what will do the serialization for use, and than call the `VersionedKeyValueToBytesStoreAdapter` that will split out raw-value, and raw-ts to call `inner.put(k,v,ts)`. Seems overly complicated. I merged 10/N already, but now I am wondering if it the right approach? Right now, we need to do this such that we can swap in/out the changeloggging store that only has a `put(k,v)` method, but why do we not add a `put(k,v,ts)` to the changeloggging (and also caching?) store and also only have `put(k,v,ts)` here? In N/10 you said that the changelogger implements `VersionedBytesStore` instead of `VersionedKeyValueStore`, and I am now questioning if it's the right call? We also add `get()` and `delete()` below without overwriting. It seems we should also add `put(k,v,ts)` in the same manner? ########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java: ########## @@ -0,0 +1,227 @@ +/* + * 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. + */ +package org.apache.kafka.streams.state.internals; + +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; + +import java.util.Objects; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.query.Position; +import org.apache.kafka.streams.query.PositionBound; +import org.apache.kafka.streams.query.Query; +import org.apache.kafka.streams.query.QueryConfig; +import org.apache.kafka.streams.query.QueryResult; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.VersionedBytesStore; +import org.apache.kafka.streams.state.VersionedKeyValueStore; +import org.apache.kafka.streams.state.VersionedRecord; + +/** + * A metered {@link VersionedKeyValueStore} wrapper that is used for recording operation + * metrics, and hence its inner {@link VersionedBytesStore} implementation does not need to provide + * its own metrics collecting functionality. The inner {@code VersionedBytesStore} of this class + * is a {@link KeyValueStore} of type <Bytes,byte[]>, so we use {@link Serde}s + * to convert from <K,ValueAndTimestamp<V>> to <Bytes,byte[]>. In particular, + * {@link NullableValueAndTimestampSerde} is used since putting a tombstone to a versioned key-value + * store requires putting a null value associated with a timestamp. + * + * @param <K> The key type + * @param <V> The (raw) value type + */ +public class MeteredVersionedKeyValueStore<K, V> + extends WrappedStateStore<VersionedBytesStore, K, V> + implements VersionedKeyValueStore<K, V> { + + private final MeteredVersionedKeyValueStoreInternal internal; + + MeteredVersionedKeyValueStore(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner); + internal = new MeteredVersionedKeyValueStoreInternal(inner, metricScope, time, keySerde, valueSerde); + } + + /** + * Private helper class which represents the functionality of a {@link VersionedKeyValueStore} + * as a {@link TimestampedKeyValueStore} so that the bulk of the metering logic may be + * inherited from {@link MeteredKeyValueStore}. As a result, the implementation of + * {@link MeteredVersionedKeyValueStore} is a simple wrapper to translate from this + * {@link TimestampedKeyValueStore} representation of a versioned key-value store into the + * {@link VersionedKeyValueStore} interface itself. + */ + private class MeteredVersionedKeyValueStoreInternal + extends MeteredKeyValueStore<K, ValueAndTimestamp<V>> + implements TimestampedKeyValueStore<K, V> { + + private final VersionedBytesStore inner; + + MeteredVersionedKeyValueStoreInternal(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner, metricScope, time, keySerde, valueSerde); + this.inner = inner; + } + + @Override + public void put(final K key, final ValueAndTimestamp<V> value) { + super.put( + key, + // versioned stores require a timestamp associated with all puts, including tombstones/deletes + value == null + ? ValueAndTimestamp.makeAllowNullable(null, context.timestamp()) + : value + ); + } + + public ValueAndTimestamp<V> get(final K key, final long asOfTimestamp) { + Objects.requireNonNull(key, "key cannot be null"); + try { + return maybeMeasureLatency(() -> outerValue(inner.get(keyBytes(key), asOfTimestamp)), time, getSensor); + } catch (final ProcessorStateException e) { + final String message = String.format(e.getMessage(), key); + throw new ProcessorStateException(message, e); + } + } + + public ValueAndTimestamp<V> delete(final K key, final long timestamp) { + Objects.requireNonNull(key, "key cannot be null"); + try { + return maybeMeasureLatency(() -> outerValue(inner.delete(keyBytes(key), timestamp)), time, deleteSensor); + } catch (final ProcessorStateException e) { + final String message = String.format(e.getMessage(), key); + throw new ProcessorStateException(message, e); + } + } + + @Override + public <R> QueryResult<R> query(final Query<R> query, + final PositionBound positionBound, + final QueryConfig config) { + final long start = time.nanoseconds(); + final QueryResult<R> result = wrapped().query(query, positionBound, config); + if (config.isCollectExecutionInfo()) { + result.addExecutionInfo( + "Handled in " + getClass() + " in " + (time.nanoseconds() - start) + "ns"); + } + // do not convert query or return types to/from inner bytes store to user-friendly types Review Comment: Not sure if I understand point (2). The implementation of a query, is bound to the _implementation_ of the store, not the store type. Thus, if there is a `UserVersionedKeyValueStore` that supports a `RangeQuery` (as example), the semantics and types are bound to `UverVersionedKeyValueStore` -- when we later add `RangeQuery` to `RocksDBVersionedStore`, it seem ok if it's different because it comes with it's own implementation? Thus I think it ok to have the code as-is going with (2) (as a matter of fact, I think we can remove this comment as it seems unnecessary to me). At least that's my understanding on IQv2. Maybe @vvcephei can chime in and verify my understanding. ########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java: ########## @@ -0,0 +1,227 @@ +/* + * 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. + */ +package org.apache.kafka.streams.state.internals; + +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; + +import java.util.Objects; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.query.Position; +import org.apache.kafka.streams.query.PositionBound; +import org.apache.kafka.streams.query.Query; +import org.apache.kafka.streams.query.QueryConfig; +import org.apache.kafka.streams.query.QueryResult; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.VersionedBytesStore; +import org.apache.kafka.streams.state.VersionedKeyValueStore; +import org.apache.kafka.streams.state.VersionedRecord; + +/** + * A metered {@link VersionedKeyValueStore} wrapper that is used for recording operation + * metrics, and hence its inner {@link VersionedBytesStore} implementation does not need to provide + * its own metrics collecting functionality. The inner {@code VersionedBytesStore} of this class + * is a {@link KeyValueStore} of type <Bytes,byte[]>, so we use {@link Serde}s + * to convert from <K,ValueAndTimestamp<V>> to <Bytes,byte[]>. In particular, + * {@link NullableValueAndTimestampSerde} is used since putting a tombstone to a versioned key-value + * store requires putting a null value associated with a timestamp. + * + * @param <K> The key type + * @param <V> The (raw) value type + */ +public class MeteredVersionedKeyValueStore<K, V> + extends WrappedStateStore<VersionedBytesStore, K, V> + implements VersionedKeyValueStore<K, V> { + + private final MeteredVersionedKeyValueStoreInternal internal; + + MeteredVersionedKeyValueStore(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner); + internal = new MeteredVersionedKeyValueStoreInternal(inner, metricScope, time, keySerde, valueSerde); + } + + /** + * Private helper class which represents the functionality of a {@link VersionedKeyValueStore} + * as a {@link TimestampedKeyValueStore} so that the bulk of the metering logic may be + * inherited from {@link MeteredKeyValueStore}. As a result, the implementation of + * {@link MeteredVersionedKeyValueStore} is a simple wrapper to translate from this + * {@link TimestampedKeyValueStore} representation of a versioned key-value store into the + * {@link VersionedKeyValueStore} interface itself. + */ + private class MeteredVersionedKeyValueStoreInternal + extends MeteredKeyValueStore<K, ValueAndTimestamp<V>> + implements TimestampedKeyValueStore<K, V> { Review Comment: Why do we not directly `extend MeteredTimestampKeyValueStore` but `extend MeteredKeyValueStore` plus `implement TimestampedKeyValueStore`? ########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java: ########## @@ -0,0 +1,227 @@ +/* + * 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. + */ +package org.apache.kafka.streams.state.internals; + +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; + +import java.util.Objects; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.query.Position; +import org.apache.kafka.streams.query.PositionBound; +import org.apache.kafka.streams.query.Query; +import org.apache.kafka.streams.query.QueryConfig; +import org.apache.kafka.streams.query.QueryResult; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.VersionedBytesStore; +import org.apache.kafka.streams.state.VersionedKeyValueStore; +import org.apache.kafka.streams.state.VersionedRecord; + +/** + * A metered {@link VersionedKeyValueStore} wrapper that is used for recording operation + * metrics, and hence its inner {@link VersionedBytesStore} implementation does not need to provide + * its own metrics collecting functionality. The inner {@code VersionedBytesStore} of this class + * is a {@link KeyValueStore} of type <Bytes,byte[]>, so we use {@link Serde}s + * to convert from <K,ValueAndTimestamp<V>> to <Bytes,byte[]>. In particular, + * {@link NullableValueAndTimestampSerde} is used since putting a tombstone to a versioned key-value + * store requires putting a null value associated with a timestamp. + * + * @param <K> The key type + * @param <V> The (raw) value type + */ +public class MeteredVersionedKeyValueStore<K, V> + extends WrappedStateStore<VersionedBytesStore, K, V> + implements VersionedKeyValueStore<K, V> { + + private final MeteredVersionedKeyValueStoreInternal internal; + + MeteredVersionedKeyValueStore(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner); + internal = new MeteredVersionedKeyValueStoreInternal(inner, metricScope, time, keySerde, valueSerde); + } + + /** + * Private helper class which represents the functionality of a {@link VersionedKeyValueStore} + * as a {@link TimestampedKeyValueStore} so that the bulk of the metering logic may be + * inherited from {@link MeteredKeyValueStore}. As a result, the implementation of + * {@link MeteredVersionedKeyValueStore} is a simple wrapper to translate from this + * {@link TimestampedKeyValueStore} representation of a versioned key-value store into the + * {@link VersionedKeyValueStore} interface itself. + */ + private class MeteredVersionedKeyValueStoreInternal + extends MeteredKeyValueStore<K, ValueAndTimestamp<V>> + implements TimestampedKeyValueStore<K, V> { + + private final VersionedBytesStore inner; + + MeteredVersionedKeyValueStoreInternal(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner, metricScope, time, keySerde, valueSerde); + this.inner = inner; + } + + @Override + public void put(final K key, final ValueAndTimestamp<V> value) { + super.put( + key, + // versioned stores require a timestamp associated with all puts, including tombstones/deletes + value == null Review Comment: Just for an internal safe-guard, we should put a `requiresNotNull` in place for this case. -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org