mjsax commented on code in PR #20403: URL: https://github.com/apache/kafka/pull/20403#discussion_r2430877769
########## streams/src/main/java/org/apache/kafka/streams/processor/internals/InitProcessorRecordContext.java: ########## @@ -0,0 +1,51 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.internals.RecordHeaders; + + +public class InitProcessorRecordContext extends ProcessorRecordContext { Review Comment: Given that we moved off the design, to have a ts that changes, but use a fixed ts now, do we still need `InitProcessorRecordContext` class? We could just create `new ProcessorRecordContext(ts, NO_OFFSET, NO_PARTITION, null, new RecordHeaders())` directly, and save this class entirely? ########## streams/src/main/java/org/apache/kafka/streams/processor/internals/InitProcessorRecordContext.java: ########## @@ -0,0 +1,51 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.internals.RecordHeaders; + + +public class InitProcessorRecordContext extends ProcessorRecordContext { + + private final long initTime; + private static final long NO_OFFSET = -1; + private static final int NO_PARTITION = -1; + + public InitProcessorRecordContext(final long currentTimestamp) { + super(ConsumerRecord.NO_TIMESTAMP, NO_OFFSET, NO_PARTITION, null, new RecordHeaders()); + this.initTime = currentTimestamp; + } + + @Override + public long timestamp() { Review Comment: I think there is not need to overwrite this method, if we just use `ProcessorContext#timestamp` ########## streams/src/main/java/org/apache/kafka/streams/processor/internals/InitProcessorRecordContext.java: ########## @@ -0,0 +1,51 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.internals.RecordHeaders; + + +public class InitProcessorRecordContext extends ProcessorRecordContext { + + private final long initTime; Review Comment: Even if we keep this class, I think we don't need `initTime`, and we can just use `ProcessorRecordContext#timestamp` ? ########## streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java: ########## @@ -1264,6 +1281,31 @@ public void process(final Record<String, String> record) { } } + private static class StatefulProcessorWithInitialization implements Processor<String, String, Void, Void> { + private KeyValueStore<String, String> store; + private final String storeName; + private final String initialKey; + private final String initialValue; + + public StatefulProcessorWithInitialization(final String storeName, final String initialKey, final String initialValue) { + this.storeName = storeName; + this.initialKey = initialKey; + this.initialValue = initialValue; + } + + @Override + public void init(final ProcessorContext<Void, Void> context) { + store = context.getStateStore(storeName); + store.put(initialKey, initialValue); + } + + @Override + public void process(final Record<String, String> record) { + store.put(record.key(), record.value()); Review Comment: nit: seems we can leave `process()` empty, as we never pipe any data through this processor? ########## streams/src/main/java/org/apache/kafka/streams/processor/internals/InitProcessorRecordContext.java: ########## @@ -0,0 +1,51 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.internals.RecordHeaders; + + +public class InitProcessorRecordContext extends ProcessorRecordContext { + + private final long initTime; + private static final long NO_OFFSET = -1; + private static final int NO_PARTITION = -1; + + public InitProcessorRecordContext(final long currentTimestamp) { + super(ConsumerRecord.NO_TIMESTAMP, NO_OFFSET, NO_PARTITION, null, new RecordHeaders()); Review Comment: ```suggestion super(currentTimestamp, NO_OFFSET, NO_PARTITION, null, new RecordHeaders()); ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
