yunfengzhou-hub commented on a change in pull request #18: URL: https://github.com/apache/flink-ml/pull/18#discussion_r727632118
########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/broadcast/operator/OneInputBroadcastWrapperOperator.java ########## @@ -0,0 +1,170 @@ +/* + * 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.flink.ml.common.broadcast.operator; + +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.ml.iteration.datacache.nonkeyed.DataCacheReader; +import org.apache.flink.ml.iteration.datacache.nonkeyed.DataCacheWriter; +import org.apache.flink.ml.iteration.datacache.nonkeyed.Segment; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; + +import org.apache.commons.collections.IteratorUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +/** Wrapper for WithBroadcastOneInputStreamOperator. */ +public class OneInputBroadcastWrapperOperator<IN, OUT> + extends AbstractBroadcastWrapperOperator<OUT, OneInputStreamOperator<IN, OUT>> + implements OneInputStreamOperator<IN, OUT> { + + private List<IN> cache; + + public OneInputBroadcastWrapperOperator( + StreamOperatorParameters<OUT> parameters, + StreamOperatorFactory<OUT> operatorFactory, + String[] broadcastStreamNames, + TypeInformation[] inTypes, + boolean[] isBlocking) { + super(parameters, operatorFactory, broadcastStreamNames, inTypes, isBlocking); + this.cache = new ArrayList<>(); + } + + @Override + public void processElement(StreamRecord<IN> streamRecord) throws Exception { + if (isBlocking[0]) { + if (areBroadcastVariablesReady()) { + for (IN ele : cache) { + wrappedOperator.processElement(new StreamRecord<>(ele)); + } + cache.clear(); + wrappedOperator.processElement(streamRecord); + + } else { + cache.add(streamRecord.getValue()); Review comment: I can see that this PR is trying to use this caching list to avoid fulfilling Flink's buffer, and the list is only stored in memory. I am worried that in case when the size of the cached records grows and exceeds the size of memory, this solution might cause java to throw exceptions and Flink job to fail. Shall we add some mechanism like follows to avoid this problem? - store part of the cached records on disk to avoid excess usage of memory. - check the size of cached records stored in memory and handle possible exceptions. ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/broadcast/operator/CacheStreamOperator.java ########## @@ -0,0 +1,141 @@ +/* + * 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.flink.ml.common.broadcast.operator; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.ml.common.broadcast.BroadcastContext; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractInput; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; +import org.apache.flink.streaming.api.operators.BoundedMultiInput; +import org.apache.flink.streaming.api.operators.Input; +import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import org.apache.commons.collections.IteratorUtils; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** The operator that process all broadcast inputs and stores them in {@link BroadcastContext}. */ +public class CacheStreamOperator<OUT> extends AbstractStreamOperatorV2<OUT> Review comment: Could it be better to rename `CacheStreamOperator` to something like `BroadcastStreamOperator`? I find it a little bit hard to associate the current name with its functionality. ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/broadcast/BroadcastContext.java ########## @@ -0,0 +1,123 @@ +/* + * 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.flink.ml.common.broadcast; + +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +public class BroadcastContext { + /** + * Store broadcast DataStreams in a Map. The key is (broadcastName, partitionId) and the value + * is (isBroaddcastVariableReady, cacheList). + */ + private static Map<Tuple2<String, Integer>, Tuple2<Boolean, List<?>>> broadcastVariables = + new HashMap<>(); + /** + * We use lock because we want to enable `getBroadcastVariable(String)` in a TM with multiple + * slots here. Note that using ConcurrentHashMap is not enough since we need "contains and get + * in an atomic operation". + */ + private static ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + public static void putBroadcastVariable( + Tuple2<String, Integer> key, Tuple2<Boolean, List<?>> variable) { + lock.writeLock().lock(); + try { + broadcastVariables.put(key, variable); + } finally { + lock.writeLock().unlock(); + } + } + + /** + * get the cached list with the given key. + * + * @param key + * @param <T> + * @return the cache list. + */ + public static <T> List<T> getBroadcastVariable(Tuple2<String, Integer> key) { + lock.readLock().lock(); + List<?> result = null; + try { + result = broadcastVariables.get(key).f1; + } finally { + lock.readLock().unlock(); + } + return (List<T>) result; + } + + /** + * get broadcast variables by name + * + * @param name + * @param <T> + * @return + */ + public static <T> List<T> getBroadcastVariable(String name) { + lock.readLock().lock(); + List<?> result = null; + try { + for (Tuple2<String, Integer> nameAndPartitionId : broadcastVariables.keySet()) { + if (name.equals(nameAndPartitionId.f0) && isCacheFinished(nameAndPartitionId)) { + result = broadcastVariables.get(nameAndPartitionId).f1; + break; Review comment: If the cached broadcast variables are the same regardless of partitionId, I personally think that it might be unnecessary to store partitionId as part of the key of `broadcastVariables`. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org