You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/05/16 01:04:45 UTC

[GitHub] [kafka] guozhangwang commented on a change in pull request #8669: MINOR: consolidate processor context for active/standby

guozhangwang commented on a change in pull request #8669:
URL: https://github.com/apache/kafka/pull/8669#discussion_r426101065



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
##########
@@ -16,87 +16,125 @@
  */
 package org.apache.kafka.streams.processor.internals;
 
-import org.apache.kafka.streams.KeyValue;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.BytesSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.internals.ApiUtils;
-import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.processor.Cancellable;
-import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.PunctuationType;
 import org.apache.kafka.streams.processor.Punctuator;
+import org.apache.kafka.streams.processor.StateRestoreCallback;
 import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.TaskId;
 import org.apache.kafka.streams.processor.To;
+import org.apache.kafka.streams.processor.internals.Task.TaskType;
 import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.apache.kafka.streams.state.KeyValueIterator;
-import org.apache.kafka.streams.state.KeyValueStore;
-import org.apache.kafka.streams.state.SessionStore;
-import org.apache.kafka.streams.state.TimestampedKeyValueStore;
-import org.apache.kafka.streams.state.TimestampedWindowStore;
-import org.apache.kafka.streams.state.ValueAndTimestamp;
-import org.apache.kafka.streams.state.WindowStore;
-import org.apache.kafka.streams.state.WindowStoreIterator;
 import org.apache.kafka.streams.state.internals.ThreadCache;
-import org.apache.kafka.streams.state.internals.WrappedStateStore;
 
 import java.time.Duration;
 import java.util.List;
 
 import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix;
+import static org.apache.kafka.streams.processor.internals.AbstractReadOnlyDecorator.getReadOnlyStore;
+import static org.apache.kafka.streams.processor.internals.AbstractReadWriteDecorator.getReadWriteStore;
 
 public class ProcessorContextImpl extends AbstractProcessorContext implements RecordCollector.Supplier {
+    public static final BytesSerializer KEY_SERIALIZER = new BytesSerializer();
+    public static final ByteArraySerializer VALUE_SERIALIZER = new ByteArraySerializer();
 
-    private final StreamTask task;
+    // The below are both null for standby tasks
+    private final StreamTask streamTask;
     private final RecordCollector collector;
+
     private final ToInternal toInternal = new ToInternal();
     private final static To SEND_TO_ALL = To.all();
 
+    final Map<String, String> storeToChangelogTopic = new HashMap<>();
+
     ProcessorContextImpl(final TaskId id,
-                         final StreamTask task,
+                         final StreamTask streamTask,
                          final StreamsConfig config,
                          final RecordCollector collector,
                          final ProcessorStateManager stateMgr,
                          final StreamsMetricsImpl metrics,
                          final ThreadCache cache) {
         super(id, config, metrics, stateMgr, cache);
-        this.task = task;
+        this.streamTask = streamTask;

Review comment:
       nit: we can add a validation that if streamTask != null then stateMgr.taskType() == ACTIVE.

##########
File path: streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java
##########
@@ -349,6 +353,27 @@ public Headers headers() {
         return recordContext.headers();
     }
 
+    @Override
+    public TaskType taskType() {
+        return TaskType.ACTIVE;
+    }
+
+    @Override
+    public void logChange(final String storeName,
+                          final Bytes key,
+                          final byte[] value,
+                          final long timestamp) {
+        recordCollector().send(

Review comment:
       When we merge the InternalMock with MockInternal would this be okay? cc @cadonna 

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java
##########
@@ -1,71 +0,0 @@
-/*
- * 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 org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.streams.processor.ProcessorContext;
-import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
-import org.apache.kafka.streams.processor.internals.RecordCollector;
-import org.apache.kafka.streams.state.StateSerdes;
-
-/**
- * Note that the use of array-typed keys is discouraged because they result in incorrect caching behavior.
- * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class,
- * i.e. use {@code StoreChangeLogger<Bytes, ...>} rather than {@code StoreChangeLogger<byte[], ...>}.
- *
- * @param <K>
- * @param <V>
- */
-class StoreChangeLogger<K, V> {
-
-    private final String topic;
-    private final int partition;
-    private final ProcessorContext context;
-    private final RecordCollector collector;
-    private final Serializer<K> keySerializer;
-    private final Serializer<V> valueSerializer;
-
-    StoreChangeLogger(final String storeName,
-                      final ProcessorContext context,
-                      final StateSerdes<K, V> serialization) {
-        this(storeName, context, context.taskId().partition, serialization);
-    }
-
-    private StoreChangeLogger(final String storeName,
-                              final ProcessorContext context,
-                              final int partition,
-                              final StateSerdes<K, V> serialization) {
-        topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName);
-        this.context = context;
-        this.partition = partition;
-        this.collector = ((RecordCollector.Supplier) context).recordCollector();
-        keySerializer = serialization.keySerializer();
-        valueSerializer = serialization.valueSerializer();
-    }
-
-    void logChange(final K key,
-                   final V value) {
-        logChange(key, value, context.timestamp());
-    }
-
-    void logChange(final K key,

Review comment:
       SGTM.




----------------------------------------------------------------
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.

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