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



##########
File path: 
flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/source/HBaseRowDataLookupFunction.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.hbase.source;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseReadWriteHelper;
+import org.apache.flink.connector.hbase.util.HBaseSerde;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+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.util.StringUtils;
+
+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.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * The HBaseRowDataLookupFunction 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 HBaseRowDataLookupFunction extends TableFunction<RowData> {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(HBaseRowDataLookupFunction.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 HBaseReadWriteHelper readHelper;
+       private transient Connection hConnection;
+       private transient HTable table;
+       private transient HBaseSerde serde;
+
+       public HBaseRowDataLookupFunction(
+                       Configuration configuration,
+                       String hTableName,
+                       HBaseTableSchema hbaseTableSchema,
+                       String nullStringLiteral) {
+               this.serializedConfig = 
HBaseConfigurationUtil.serializeConfiguration(configuration);
+               this.hTableName = hTableName;
+               this.hbaseTableSchema = hbaseTableSchema;
+               this.nullStringLiteral = nullStringLiteral;
+       }
+
+       /**
+        * The invoke entry point of lookup function.
+        * @param rowKey the lookup key. Currently only support single rowkey.
+        */
+       public void eval(Object rowKey) throws IOException {
+               // fetch result
+               Result result = table.get(readHelper.createGet(rowKey));

Review comment:
       Do not use `HBaseReadWriteHelper`. It uses legacy type. You can add a 
new method `createGet` in `HbaseSerDe`. 

##########
File path: 
flink-connectors/flink-connector-hbase/src/test/java/org/apache/flink/connector/hbase/HBaseConnectorITCase.java
##########
@@ -558,71 +539,71 @@ public void testHBaseLookupTableSource() throws Exception 
{
                }
                StreamExecutionEnvironment streamEnv = 
StreamExecutionEnvironment.getExecutionEnvironment();
                StreamTableEnvironment streamTableEnv = 
StreamTableEnvironment.create(streamEnv, streamSettings);
+
+               // prepare dimension table data.
+               DataStream<Row> ds = 
streamEnv.fromCollection(testData1).returns(testTypeInfo1);
+               streamTableEnv.createTemporaryView("testData", ds);
+               String ddl = getDDLForTestTable3();
+               streamTableEnv.executeSql(ddl);
+
+               String query = "INSERT INTO hbase " +
+                       "SELECT rowkey, ROW(f1c1), ROW(f2c1, f2c2), ROW(f3c1, 
f3c2, f3c3), ROW(f4c1, f4c2, f4c3) " +
+                       "FROM testData";
+               TableEnvUtil.execInsertSqlAndWaitResult(streamTableEnv, query);
                StreamITCase.clear();
 
                // prepare a source table
                String srcTableName = "src";
-               DataStream<Row> ds = 
streamEnv.fromCollection(testData2).returns(testTypeInfo2);
-               Table in = streamTableEnv.fromDataStream(ds, $("a"), $("b"), 
$("c"), $("proc").proctime());
+               DataStream<Row> srcDs = 
streamEnv.fromCollection(testData2).returns(testTypeInfo2);
+               Table in = streamTableEnv.fromDataStream(srcDs, $("a"), $("b"), 
$("c"), $("proc").proctime());
                streamTableEnv.registerTable(srcTableName, in);
 
-               if (isLegacyConnector) {
-                       Map<String, String> tableProperties = 
hbaseTableProperties();
-                       TableSource<?> source = TableFactoryService
-                               .find(HBaseTableFactory.class, tableProperties)
-                               .createTableSource(tableProperties);
-                       ((TableEnvironmentInternal) 
streamTableEnv).registerTableSourceInternal("hbaseLookup", source);
-               } else {
-                       streamTableEnv.executeSql(
-                                       "CREATE TABLE hbaseLookup (" +
-                                       " family1 ROW<col1 INT>," +
-                                       " rk INT," +
-                                       " family2 ROW<col1 STRING, col2 
BIGINT>," +
-                                       " family3 ROW<col1 DOUBLE, col2 
BOOLEAN, col3 STRING>" +
-                                       ") WITH (" +
-                                       " 'connector' = 'hbase-1.4'," +
-                                       " 'table-name' = '" + TEST_TABLE_1 + 
"'," +
-                                       " 'zookeeper.quorum' = '" + 
getZookeeperQuorum() + "'" +
-                                       ")");
-               }
                // perform a temporal table join query
-               String query = "SELECT a,family1.col1, family3.col3 FROM src " +
-                       "JOIN hbaseLookup FOR SYSTEM_TIME AS OF src.proc as h 
ON src.a = h.rk";
-               Table result = streamTableEnv.sqlQuery(query);
+               String dimJoinQuery = "SELECT" +
+                       " a," +
+                       " b," +
+                       " family1.col1," +
+                       " family2.col1," +
+                       " family2.col2," +
+                       " family3.col1," +
+                       " family3.col2," +
+                       " family3.col3," +
+                       " family4.col1," +
+                       " family4.col2," +
+                       " family4.col3" +
+                       " FROM src JOIN hbase FOR SYSTEM_TIME AS OF src.proc as 
h ON src.a = h.rowkey";
+               Iterator<Row> collected = 
streamTableEnv.executeSql(dimJoinQuery).collect();
+               List<String> result = Lists.newArrayList(collected).stream()
+                       .map(Row::toString)
+                       .sorted()
+                       .collect(Collectors.toList());
+
+               // check result, the time type in collected result is 
LOCAL_DATE_TIME, LOCAL_DATE, LOCAL_TIME
+               List<String> expected = new ArrayList<>();
+               
expected.add("1,1,10,Hello-1,100,1.01,false,Welt-1,2019-08-18T19:00,2019-08-18,19:00");
+               
expected.add("2,2,20,Hello-2,200,2.02,true,Welt-2,2019-08-18T19:01,2019-08-18,19:01");
+               
expected.add("3,2,30,Hello-3,300,3.03,false,Welt-3,2019-08-18T19:02,2019-08-18,19:02");
+               
expected.add("3,3,30,Hello-3,300,3.03,false,Welt-3,2019-08-18T19:02,2019-08-18,19:02");

Review comment:
       Could you add a DECIMAL to the lookup table? It is the other error-prone 
type. 




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