wuchong commented on a change in pull request #14684:
URL: https://github.com/apache/flink/pull/14684#discussion_r592279152



##########
File path: 
flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.connector.hbase2.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.hbase.options.HBaseLookupOptions;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseSerde;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.AsyncTableFunction;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The HBaseRowDataAsyncLookupFunction is a standard user-defined table 
function, it can be used in
+ * tableAPI and also useful for temporal table join plan in SQL. It looks up 
the result as {@link
+ * RowData}.
+ */
+@Internal
+public class HBaseRowDataAsyncLookupFunction extends 
AsyncTableFunction<RowData> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(HBaseRowDataAsyncLookupFunction.class);
+    private static final long serialVersionUID = 1L;
+
+    private final String hTableName;
+    private final byte[] serializedConfig;
+    private final HBaseTableSchema hbaseTableSchema;
+    private final String nullStringLiteral;
+
+    private transient AsyncConnection asyncConnection;
+    private transient AsyncTable<ScanResultConsumer> table;
+    private transient HBaseSerde serde;
+
+    private final long cacheMaxSize;
+    private final long cacheExpireMs;
+    private final int maxRetryTimes;
+    private transient Cache<Object, RowData> cache;
+
+    public HBaseRowDataAsyncLookupFunction(
+            Configuration configuration,
+            String hTableName,
+            HBaseTableSchema hbaseTableSchema,
+            String nullStringLiteral, HBaseLookupOptions lookupOptions) {
+        this.serializedConfig = 
HBaseConfigurationUtil.serializeConfiguration(configuration);
+        this.hTableName = hTableName;
+        this.hbaseTableSchema = hbaseTableSchema;
+        this.nullStringLiteral = nullStringLiteral;
+        this.cacheMaxSize = lookupOptions.getCacheMaxSize();
+        this.cacheExpireMs = lookupOptions.getCacheExpireMs();
+        this.maxRetryTimes = lookupOptions.getMaxRetryTimes();
+    }
+
+    @Override
+    public void open(FunctionContext context) {
+        LOG.info("start open ...");
+        Configuration config = prepareRuntimeConfiguration();
+        CompletableFuture<AsyncConnection> asyncConnectionFuture = 
ConnectionFactory.createAsyncConnection(config);
+        try {
+            asyncConnection = asyncConnectionFuture.get();
+            table = asyncConnection.getTable(TableName.valueOf(hTableName), 
(ExecutorService) Executors.directExecutor());

Review comment:
       The CPU cores may very large depends on the matchine. I would suggest to 
use a hard code 16 as the example. We can parameterize it in the future if 
needed. 




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to