JingsongLi commented on code in PR #281:
URL: https://github.com/apache/flink-table-store/pull/281#discussion_r963325253


##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/lookup/FileStoreLookupFunction.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.store.connector.lookup;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.predicate.Predicate;
+import org.apache.flink.table.store.file.predicate.PredicateFilter;
+import org.apache.flink.table.store.file.schema.TableSchema;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.table.source.TableStreamingReader;
+import org.apache.flink.table.store.utils.TypeUtils;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FileUtils;
+
+import org.apache.flink.shaded.guava30.com.google.common.primitives.Ints;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.flink.table.store.connector.RocksDBOptions.LRU_CACHE_MAX_ROWS;
+import static 
org.apache.flink.table.store.file.predicate.PredicateBuilder.transformFieldMapping;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** A lookup {@link TableFunction} for file store. */
+public class FileStoreLookupFunction extends TableFunction<RowData> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FileStoreLookupFunction.class);
+
+    private final FileStoreTable table;
+    private final List<String> projectFields;
+    private final List<String> joinKeys;
+    @Nullable private final Predicate predicate;
+
+    private transient Duration refreshInterval;
+    private transient File path;
+    private transient RocksDBStateFactory stateFactory;
+    private transient LookupTable lookupTable;
+
+    // timestamp when cache expires
+    private transient long nextLoadTime;
+    private transient TableStreamingReader streamingReader;
+
+    public FileStoreLookupFunction(
+            FileStoreTable table,
+            int[] projection,
+            int[] joinKeyIndex,
+            @Nullable Predicate predicate) {
+        TableSchema schema = table.schema();
+        checkArgument(
+                schema.partitionKeys().isEmpty(), "Currently only support 
non-partitioned table.");
+        checkArgument(schema.primaryKeys().size() > 0, "Currently only support 
primary key table.");
+        this.table = table;
+
+        // join keys are based on projection fields
+        this.joinKeys =
+                Arrays.stream(joinKeyIndex)
+                        .mapToObj(i -> schema.fieldNames().get(projection[i]))
+                        .collect(Collectors.toList());
+
+        this.projectFields =
+                Arrays.stream(projection)
+                        .mapToObj(i -> schema.fieldNames().get(i))
+                        .collect(Collectors.toList());
+
+        // add primary keys
+        for (String field : schema.primaryKeys()) {
+            if (!projectFields.contains(field)) {
+                projectFields.add(field);
+            }
+        }
+
+        this.predicate = predicate;
+    }
+
+    @Override
+    public void open(FunctionContext context) throws Exception {
+        super.open(context);
+        String tmpDirectory = getTmpDirectory(context);

Review Comment:
   No, this is not java temp directory, this is work directory of Flink. State 
will use this directory too.



-- 
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

Reply via email to