SmirAlex commented on code in PR #20447: URL: https://github.com/apache/flink/pull/20447#discussion_r941497728
########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/lookup/fullcache/inputformat/InputFormatCacheLoader.java: ########## @@ -0,0 +1,159 @@ +/* + * 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.table.runtime.functions.table.lookup.fullcache.inputformat; + +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.functions.table.lookup.fullcache.CacheLoader; +import org.apache.flink.table.runtime.keyselector.GenericRowDataKeySelector; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +/** {@link CacheLoader} that used {@link InputFormat} for loading data into the cache. */ +public class InputFormatCacheLoader extends CacheLoader { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(InputFormatCacheLoader.class); + + private final InputFormat<RowData, InputSplit> initialInputFormat; + private final GenericRowDataKeySelector keySelector; + private final RowDataSerializer cacheEntriesSerializer; + + private transient volatile List<InputSplitCacheLoadTask> cacheLoadTasks; + private transient Configuration parameters; + + private volatile boolean isStopped; + + public InputFormatCacheLoader( + InputFormat<RowData, ?> initialInputFormat, + GenericRowDataKeySelector keySelector, + RowDataSerializer cacheEntriesSerializer) { + // noinspection unchecked + this.initialInputFormat = (InputFormat<RowData, InputSplit>) initialInputFormat; + this.keySelector = keySelector; + this.cacheEntriesSerializer = cacheEntriesSerializer; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + this.parameters = parameters; + this.initialInputFormat.configure(parameters); + } + + @Override + protected void reloadCache() throws Exception { + InputSplit[] inputSplits = createInputSplits(); + int numSplits = inputSplits.length; + // load data into the another copy of cache + // notice: it requires twice more memory, but on the other hand we don't need any blocking + // cache has default initialCapacity and loadFactor, but overridden concurrencyLevel + ConcurrentHashMap<RowData, Collection<RowData>> newCache = + new ConcurrentHashMap<>(16, 0.75f, getConcurrencyLevel(numSplits)); + this.cacheLoadTasks = + Arrays.stream(inputSplits) + .map(split -> createCacheLoadTask(split, newCache)) + .collect(Collectors.toList()); + + // run first task and start numTasks - 1 threads to run remaining tasks + ExecutorService cacheLoadTaskService = null; + List<Future<?>> futures = null; + if (numSplits > 1) { + futures = new ArrayList<>(); + int numThreads = getConcurrencyLevel(numSplits) - 1; + cacheLoadTaskService = Executors.newFixedThreadPool(numThreads); + for (int i = 1; i < numSplits; i++) { + Future<?> future = cacheLoadTaskService.submit(cacheLoadTasks.get(i)); + futures.add(future); + } + } + cacheLoadTasks.get(0).run(); + if (cacheLoadTaskService != null) { + for (Future<?> future : futures) { + future.get(); // if any exception occurs it will be thrown here Review Comment: We have simple futures here, not CompletableFutures. -- 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]
