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 2021/08/17 19:02:48 UTC

[GitHub] [kafka] vvcephei commented on a change in pull request #11213: KAFKA-13201: Convert KTable aggregations to new PAPI

vvcephei commented on a change in pull request #11213:
URL: https://github.com/apache/kafka/pull/11213#discussion_r690642563



##########
File path: streams/src/test/java/org/apache/kafka/test/MockInternalNewProcessorContext.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.test;
+
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.MockProcessorContext;
+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.api.Record;
+import org.apache.kafka.streams.processor.api.RecordMetadata;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.ProcessorNode;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.RecordCollector;
+import org.apache.kafka.streams.processor.internals.StreamTask;
+import org.apache.kafka.streams.processor.internals.Task.TaskType;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.internals.ThreadCache;
+import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener;
+
+import java.io.File;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+public class MockInternalNewProcessorContext<KOut, VOut> extends MockProcessorContext implements InternalProcessorContext<KOut, VOut> {

Review comment:
       It looks like this is copied and modified from https://github.com/apache/kafka/blob/trunk/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java . Is that right?
   
   We also have https://github.com/apache/kafka/blob/trunk/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java , and it's been a longstanding thorn in our side that there's a proliferation of these context implementations.
   
   I'm hoping that by the time we're done with all these migrations, we can actually converge on this new class and delete the other two.

##########
File path: streams/src/test/java/org/apache/kafka/test/MockInternalNewProcessorContext.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.test;
+
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.MockProcessorContext;

Review comment:
       This is the one defined in `test-utils`, right? Should we be using the new PAPI one (https://github.com/apache/kafka/blob/trunk/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java) instead?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java
##########
@@ -150,16 +153,16 @@ public void init(final org.apache.kafka.streams.processor.ProcessorContext conte
         }
 
         @Override
-        public void process(final K key, final Change<V> value) {
+        public void process(final Record<K, Change<V>> record) {
             observedStreamTime = Math.max(observedStreamTime, internalProcessorContext.timestamp());
-            buffer(key, value);
+            buffer(record);
             enforceConstraints();
         }
 
-        private void buffer(final K key, final Change<V> value) {
-            final long bufferTime = bufferTimeDefinition.time(internalProcessorContext, key);
+        private void buffer(final Record<K, Change<V>> record) {
+            final long bufferTime = bufferTimeDefinition.time(internalProcessorContext, record.key());
 
-            buffer.put(bufferTime, key, value, internalProcessorContext.recordContext());
+            buffer.put(bufferTime, record.key(), record.value(), internalProcessorContext.recordContext());

Review comment:
       It seems like these calls could be updated to use the `Record` itself instead of the key, value, and `InternalProcesorContext#recordContext`.




-- 
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-unsubscribe@kafka.apache.org

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