stevenzwu commented on code in PR #6382: URL: https://github.com/apache/iceberg/pull/6382#discussion_r1143943999
########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java: ########## @@ -0,0 +1,134 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatistics; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatisticsFactory; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; + +/** + * DataStatisticsOperator can help to improve data clustering based on the key. Review Comment: nit: this sentence is not accurate. statistics operator itself doesn't help improve data clustering. we can probably remove this sentence as the following sentence seems good as a short summary sentence. ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/statistics/MapDataStatistics.java: ########## @@ -0,0 +1,61 @@ +/* + * 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.iceberg.flink.sink.shuffle.statistics; + +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +/** MapDataStatistics uses map to count key frequency */ +@Internal +public class MapDataStatistics<K> implements DataStatistics<K> { + private final Map<K, Long> statistics = Maps.newHashMap(); + + @Override + public boolean isEmpty() { + return statistics.size() == 0; + } + + @Override + public void add(K key) { + // increase count of occurrence by one in the dataStatistics map + statistics.merge(key, 1L, Long::sum); + } + + @Override + public void merge(DataStatistics<K> otherStatistics) { + Preconditions.checkArgument( + otherStatistics instanceof MapDataStatistics, + "Can not merge this type of statistics: " + otherStatistics); Review Comment: nit: the error msg can be a little clearer. note that maybe we can add a `String type()` to the `DataStatistics` interface. we shouldn't dump the whole statistics from toString() in the error msg. ``` Map statistics can not merge with statistics type: " + otherStatistics.type() ``` ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java: ########## @@ -0,0 +1,134 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatistics; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatisticsFactory; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; + +/** + * DataStatisticsOperator can help to improve data clustering based on the key. + * + * <p>DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be + * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to + * shuffle record to improve data clustering while maintaining relative balanced traffic + * distribution to downstream subtasks. + */ +class DataStatisticsOperator<T, K> extends AbstractStreamOperator<DataStatisticsOrRecord<T, K>> + implements OneInputStreamOperator<T, DataStatisticsOrRecord<T, K>>, OperatorEventHandler { + private static final long serialVersionUID = 1L; + + // keySelector will be used to generate key from data for collecting data statistics + private final KeySelector<T, K> keySelector; + private final OperatorEventGateway operatorEventGateway; + private final DataStatisticsFactory<K> statisticsFactory; + private DataStatistics<K> localStatistics; Review Comment: The two statistics variables probably need to be `transient volatile`. The other `globalStatisticsState` need to be `transient` ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java: ########## @@ -0,0 +1,134 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatistics; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatisticsFactory; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; + +/** + * DataStatisticsOperator can help to improve data clustering based on the key. + * + * <p>DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be + * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to + * shuffle record to improve data clustering while maintaining relative balanced traffic + * distribution to downstream subtasks. + */ +class DataStatisticsOperator<T, K> extends AbstractStreamOperator<DataStatisticsOrRecord<T, K>> + implements OneInputStreamOperator<T, DataStatisticsOrRecord<T, K>>, OperatorEventHandler { + private static final long serialVersionUID = 1L; + + // keySelector will be used to generate key from data for collecting data statistics + private final KeySelector<T, K> keySelector; + private final OperatorEventGateway operatorEventGateway; + private final DataStatisticsFactory<K> statisticsFactory; + private DataStatistics<K> localStatistics; + private DataStatistics<K> globalStatistics; + private ListState<DataStatistics<K>> globalStatisticsState; + + public DataStatisticsOperator( + KeySelector<T, K> keySelector, + OperatorEventGateway operatorEventGateway, + DataStatisticsFactory<K> statisticsFactory) { + this.keySelector = keySelector; + this.operatorEventGateway = operatorEventGateway; + this.statisticsFactory = statisticsFactory; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + localStatistics = statisticsFactory.createDataStatistics(); + globalStatisticsState = + context + .getOperatorStateStore() + .getUnionListState( + new ListStateDescriptor<>( + "globalStatisticsState", + TypeInformation.of(new TypeHint<DataStatistics<K>>() {}))); + + if (context.isRestored()) { + int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + if (globalStatisticsState.get() == null + || !globalStatisticsState.get().iterator().hasNext()) { + LOG.warn("Subtask {} doesn't have global statistics state to restore", subtaskIndex); + globalStatistics = statisticsFactory.createDataStatistics(); + } else { + LOG.info("Restoring global statistics state for subtask {}", subtaskIndex); + globalStatistics = globalStatisticsState.get().iterator().next(); + } + } else { + globalStatistics = statisticsFactory.createDataStatistics(); + } + } + + @Override + public void open() throws Exception { + if (!globalStatistics.isEmpty()) { + output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + } + } + + @Override + public void handleOperatorEvent(OperatorEvent evt) { + // TODO: receive event with aggregated statistics from coordinator and update globalStatistics + } + + @Override + public void processElement(StreamRecord<T> streamRecord) throws Exception { + final K key = keySelector.getKey(streamRecord.getValue()); + localStatistics.add(key); + output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(streamRecord.getValue()))); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + long checkpointId = context.getCheckpointId(); + LOG.debug("Taking data statistics operator snapshot for checkpoint {}", checkpointId); + + // Update globalStatisticsState with latest global statistics + if (!globalStatistics.isEmpty()) { + globalStatisticsState.clear(); + globalStatisticsState.add(globalStatistics); Review Comment: with union state, we only subtask 0 to checkpoint the state. ########## flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java: ########## @@ -0,0 +1,155 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.TestTaskStateManager; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatisticsFactory; +import org.apache.iceberg.flink.sink.shuffle.statistics.MapDataStatistics; +import org.apache.iceberg.flink.sink.shuffle.statistics.MapDataStatisticsFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestDataStatisticsOperator { + private DataStatisticsOperator<String, String> operator; + + private Environment getTestingEnvironment() { + return new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); + } + + @Before + public void before() throws Exception { + MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); + KeySelector<String, String> keySelector = + new KeySelector<String, String>() { + private static final long serialVersionUID = 7662520075515707428L; + + @Override + public String getKey(String value) { + return value; + } + }; + DataStatisticsFactory<String> dataStatisticsFactory = new MapDataStatisticsFactory<>(); + + this.operator = new DataStatisticsOperator<>(keySelector, mockGateway, dataStatisticsFactory); + Environment env = getTestingEnvironment(); + this.operator.setup( + new OneInputStreamTask<String, String>(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + } + + @After + public void clean() throws Exception { + operator.close(); + } + + @Test + public void testProcessElement() throws Exception { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + operator.processElement(new StreamRecord<>("a")); + operator.processElement(new StreamRecord<>("a")); + operator.processElement(new StreamRecord<>("b")); + assertTrue(operator.localDataStatistics() instanceof MapDataStatistics); + MapDataStatistics<String> mapDataStatistics = + (MapDataStatistics<String>) operator.localDataStatistics(); + assertTrue(mapDataStatistics.dataStatistics().containsKey("a")); + assertTrue(mapDataStatistics.dataStatistics().containsKey("b")); + assertEquals(2L, (long) mapDataStatistics.dataStatistics().get("a")); + assertEquals(1L, (long) mapDataStatistics.dataStatistics().get("b")); + } + + @Test + public void testOperatorOutput() throws Exception { + try (OneInputStreamOperatorTestHarness<String, DataStatisticsOrRecord<String, String>> + testHarness = createHarness(this.operator)) { + testHarness.processElement(new StreamRecord<>("a")); + testHarness.processElement(new StreamRecord<>("b")); + testHarness.processElement(new StreamRecord<>("b")); + + List<String> recordsOutput = + testHarness.extractOutputValues().stream() + .filter(DataStatisticsOrRecord::hasRecord) + .map(DataStatisticsOrRecord::record) + .collect(Collectors.toList()); + assertThat(recordsOutput) + .containsExactlyInAnyOrderElementsOf(ImmutableList.of("a", "b", "b")); + } + } + + private StateInitializationContext getStateContext() throws Exception { + // Create the state context. + OperatorStateStore operatorStateStore = createOperatorStateStore(); + return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + } + + private OperatorStateStore createOperatorStateStore() throws Exception { Review Comment: nit: should this be merged with the method above? ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java: ########## @@ -0,0 +1,82 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.io.Serializable; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatistics; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * The wrapper class for data statistics and record. It is the only way for data statistics operator to send + * global data statistics to custom partitioner to distribute data based on statistics + * + * <p>DataStatisticsOrRecord is sent from {@link DataStatisticsOperator} to partitioner. It Review Comment: it seems that we can remove the comments before `It contains either ...`. We can also change `It contains either ...` to `This wrapper class contains either ...` ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/statistics/MapDataStatistics.java: ########## @@ -0,0 +1,61 @@ +/* + * 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.iceberg.flink.sink.shuffle.statistics; + +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +/** MapDataStatistics uses map to count key frequency */ +@Internal +public class MapDataStatistics<K> implements DataStatistics<K> { + private final Map<K, Long> statistics = Maps.newHashMap(); + + @Override + public boolean isEmpty() { + return statistics.size() == 0; + } + + @Override + public void add(K key) { + // increase count of occurrence by one in the dataStatistics map + statistics.put(key, statistics.getOrDefault(key, 0L) + 1L); + } + + @Override + public void merge(DataStatistics<K> otherStatistics) { + Preconditions.checkArgument( + otherStatistics instanceof MapDataStatistics, + "Can not merge this type of statistics: " + otherStatistics); + MapDataStatistics<K> mapDataStatistic = (MapDataStatistics<K>) otherStatistics; Review Comment: I agree with @yegangy0718 that it is a little better to check the type IllegalArgumentException ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java: ########## @@ -0,0 +1,134 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatistics; +import org.apache.iceberg.flink.sink.shuffle.statistics.DataStatisticsFactory; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; + +/** + * DataStatisticsOperator can help to improve data clustering based on the key. + * + * <p>DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be + * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to + * shuffle record to improve data clustering while maintaining relative balanced traffic + * distribution to downstream subtasks. + */ +class DataStatisticsOperator<T, K> extends AbstractStreamOperator<DataStatisticsOrRecord<T, K>> + implements OneInputStreamOperator<T, DataStatisticsOrRecord<T, K>>, OperatorEventHandler { + private static final long serialVersionUID = 1L; + + // keySelector will be used to generate key from data for collecting data statistics + private final KeySelector<T, K> keySelector; + private final OperatorEventGateway operatorEventGateway; + private final DataStatisticsFactory<K> statisticsFactory; + private DataStatistics<K> localStatistics; + private DataStatistics<K> globalStatistics; + private ListState<DataStatistics<K>> globalStatisticsState; + + public DataStatisticsOperator( + KeySelector<T, K> keySelector, + OperatorEventGateway operatorEventGateway, + DataStatisticsFactory<K> statisticsFactory) { + this.keySelector = keySelector; + this.operatorEventGateway = operatorEventGateway; + this.statisticsFactory = statisticsFactory; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + localStatistics = statisticsFactory.createDataStatistics(); + globalStatisticsState = + context + .getOperatorStateStore() + .getUnionListState( + new ListStateDescriptor<>( + "globalStatisticsState", + TypeInformation.of(new TypeHint<DataStatistics<K>>() {}))); + + if (context.isRestored()) { + int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + if (globalStatisticsState.get() == null + || !globalStatisticsState.get().iterator().hasNext()) { + LOG.warn("Subtask {} doesn't have global statistics state to restore", subtaskIndex); + globalStatistics = statisticsFactory.createDataStatistics(); + } else { + LOG.info("Restoring global statistics state for subtask {}", subtaskIndex); + globalStatistics = globalStatisticsState.get().iterator().next(); + } + } else { + globalStatistics = statisticsFactory.createDataStatistics(); + } + } + + @Override + public void open() throws Exception { + if (!globalStatistics.isEmpty()) { + output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + } + } + + @Override + public void handleOperatorEvent(OperatorEvent evt) { + // TODO: receive event with aggregated statistics from coordinator and update globalStatistics + } + + @Override + public void processElement(StreamRecord<T> streamRecord) throws Exception { + final K key = keySelector.getKey(streamRecord.getValue()); + localStatistics.add(key); + output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(streamRecord.getValue()))); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + long checkpointId = context.getCheckpointId(); + LOG.debug("Taking data statistics operator snapshot for checkpoint {}", checkpointId); Review Comment: this line can be info level since it is infrequent and small. this line should also log the subtask id too. We can add another debug or trace level logging that dumps the actual statistics. -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
