leonardBang commented on a change in pull request #13128: URL: https://github.com/apache/flink/pull/13128#discussion_r491940668
########## File path: flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/AbstractHBaseValidator.java ########## @@ -44,20 +43,26 @@ public void validate(DescriptorProperties properties) { super.validate(properties); properties.validateValue(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_HBASE, false); properties.validateString(CONNECTOR_TABLE_NAME, false, 1); - properties.validateString(CONNECTOR_ZK_QUORUM, false, 1); + properties.validateString(CONNECTOR_ZK_QUORUM, validateZkQuorum(), 1); properties.validateString(CONNECTOR_ZK_NODE_PARENT, true, 1); validateSinkProperties(properties); validateVersion(properties); } + protected abstract boolean validateZkQuorum(); Review comment: minor: How about renaming to `zkQuorumIsOptional()`? ########## File path: flink-connectors/flink-connector-hbase-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory ########## @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.connector.hbase.HBaseDynamicTableFactory Review comment: I think we should distinguish the `HBase1DynamicTableFactory` and `HBase2DynamicTableFactory` to ensure user can use the two connector at the same time. And the class' path in hbase2 module and hbase1 module also need to be distinguished to avoid class conflicts. ########## File path: flink-connectors/flink-connector-hbase-1.4/pom.xml ########## @@ -0,0 +1,287 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- +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. +--> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> + + <modelVersion>4.0.0</modelVersion> + + <parent> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connectors</artifactId> + <version>1.12-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>flink-connector-hbase-1.4_${scala.binary.version}</artifactId> Review comment: I think the hbase sql-connector also need to adjust. [1]https://github.com/apache/flink/blob/master/flink-connectors/flink-sql-connector-hbase/pom.xml#L39 And do we support to hbase sql-connector 2 in this PR,I'm ok to support it in another PR, there will be some dependency problem to deal. ########## File path: flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase/HBaseConnectorITCase.java ########## @@ -0,0 +1,591 @@ +/* + * 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; + +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.connector.hbase.source.HBaseInputFormat; +import org.apache.flink.connector.hbase.source.HBaseTableSource; +import org.apache.flink.connector.hbase.util.HBaseTestBase; +import org.apache.flink.connector.hbase.util.PlannerType; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.BatchTableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentInternal; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.hbase.util.PlannerType.OLD_PLANNER; +import static org.apache.flink.table.api.Expressions.$; +import static org.junit.Assert.assertEquals; + +/** + * IT cases for HBase connector (including HBaseTableSource and HBaseTableSink). + */ +@RunWith(Parameterized.class) +public class HBaseConnectorITCase extends HBaseTestBase { + + @Parameterized.Parameter + public PlannerType planner; + + @Parameterized.Parameter(1) + public boolean isLegacyConnector; + + @Override + protected PlannerType planner() { + return planner; + } + + @Parameterized.Parameters(name = "planner = {0}, legacy = {1}") + public static Object[] parameters() { + return new Object[][]{ + new Object[]{PlannerType.BLINK_PLANNER, true}, + new Object[]{PlannerType.BLINK_PLANNER, false}, + new Object[]{PlannerType.OLD_PLANNER, true} + }; + } + + // ------------------------------------------------------------------------------------- + // HBaseTableSource tests + // ------------------------------------------------------------------------------------- + + @Test + public void testTableSourceFullScan() { + TableEnvironment tEnv = createBatchTableEnv(); + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + hbaseTable.setRowKey(ROW_KEY, Integer.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW<col1 INT>," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT " + + " h.family1.col1, " + + " h.family2.col1, " + + " h.family2.col2, " + + " h.family3.col1, " + + " h.family3.col2, " + + " h.family3.col3 " + + "FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "10,Hello-1,100,1.01,false,Welt-1\n" + + "20,Hello-2,200,2.02,true,Welt-2\n" + + "30,Hello-3,300,3.03,false,Welt-3\n" + + "40,null,400,4.04,true,Welt-4\n" + + "50,Hello-5,500,5.05,false,Welt-5\n" + + "60,Hello-6,600,6.06,true,Welt-6\n" + + "70,Hello-7,700,7.07,false,Welt-7\n" + + "80,null,800,8.08,true,Welt-8\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test + public void testTableSourceProjection() { + TableEnvironment tEnv = createBatchTableEnv(); + + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + hbaseTable.setRowKey(ROW_KEY, Integer.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW<col1 INT>," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT " + + " h.family1.col1, " + + " h.family3.col1, " + + " h.family3.col2, " + + " h.family3.col3 " + + "FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "10,1.01,false,Welt-1\n" + + "20,2.02,true,Welt-2\n" + + "30,3.03,false,Welt-3\n" + + "40,4.04,true,Welt-4\n" + + "50,5.05,false,Welt-5\n" + + "60,6.06,true,Welt-6\n" + + "70,7.07,false,Welt-7\n" + + "80,8.08,true,Welt-8\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test + public void testTableSourceFieldOrder() { + TableEnvironment tEnv = createBatchTableEnv(); + + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + // shuffle order of column registration + hbaseTable.setRowKey(ROW_KEY, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " rowkey INT PRIMARY KEY," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " family1 ROW<col1 INT>" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "1,Hello-1,100,1.01,false,Welt-1,10\n" + + "2,Hello-2,200,2.02,true,Welt-2,20\n" + + "3,Hello-3,300,3.03,false,Welt-3,30\n" + + "4,null,400,4.04,true,Welt-4,40\n" + + "5,Hello-5,500,5.05,false,Welt-5,50\n" + + "6,Hello-6,600,6.06,true,Welt-6,60\n" + + "7,Hello-7,700,7.07,false,Welt-7,70\n" + + "8,null,800,8.08,true,Welt-8,80\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test Review comment: Could we add testTableSourceWithTableAPI here when rebase FLINK-18604? ########## File path: flink-connectors/flink-connector-hbase-1.4/pom.xml ########## @@ -0,0 +1,287 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- +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. +--> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> + + <modelVersion>4.0.0</modelVersion> + + <parent> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connectors</artifactId> + <version>1.12-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>flink-connector-hbase-1.4_${scala.binary.version}</artifactId> Review comment: I think the hbase sql-connector also need to adjust. [1]https://github.com/apache/flink/blob/master/flink-connectors/flink-sql-connector-hbase/pom.xml#L39 And do we support to hbase sql-connector 2 in this PR? I'm ok to support it in another PR, there will be some dependency problem to deal. ########## File path: flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase/HBaseConnectorITCase.java ########## @@ -0,0 +1,591 @@ +/* + * 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; + +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.connector.hbase.source.HBaseInputFormat; +import org.apache.flink.connector.hbase.source.HBaseTableSource; +import org.apache.flink.connector.hbase.util.HBaseTestBase; +import org.apache.flink.connector.hbase.util.PlannerType; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.BatchTableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentInternal; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.hbase.util.PlannerType.OLD_PLANNER; +import static org.apache.flink.table.api.Expressions.$; +import static org.junit.Assert.assertEquals; + +/** + * IT cases for HBase connector (including HBaseTableSource and HBaseTableSink). + */ +@RunWith(Parameterized.class) +public class HBaseConnectorITCase extends HBaseTestBase { + + @Parameterized.Parameter + public PlannerType planner; + + @Parameterized.Parameter(1) + public boolean isLegacyConnector; + + @Override + protected PlannerType planner() { + return planner; + } + + @Parameterized.Parameters(name = "planner = {0}, legacy = {1}") + public static Object[] parameters() { + return new Object[][]{ + new Object[]{PlannerType.BLINK_PLANNER, true}, + new Object[]{PlannerType.BLINK_PLANNER, false}, + new Object[]{PlannerType.OLD_PLANNER, true} + }; + } + + // ------------------------------------------------------------------------------------- + // HBaseTableSource tests + // ------------------------------------------------------------------------------------- + + @Test + public void testTableSourceFullScan() { + TableEnvironment tEnv = createBatchTableEnv(); + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + hbaseTable.setRowKey(ROW_KEY, Integer.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW<col1 INT>," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT " + + " h.family1.col1, " + + " h.family2.col1, " + + " h.family2.col2, " + + " h.family3.col1, " + + " h.family3.col2, " + + " h.family3.col3 " + + "FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "10,Hello-1,100,1.01,false,Welt-1\n" + + "20,Hello-2,200,2.02,true,Welt-2\n" + + "30,Hello-3,300,3.03,false,Welt-3\n" + + "40,null,400,4.04,true,Welt-4\n" + + "50,Hello-5,500,5.05,false,Welt-5\n" + + "60,Hello-6,600,6.06,true,Welt-6\n" + + "70,Hello-7,700,7.07,false,Welt-7\n" + + "80,null,800,8.08,true,Welt-8\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test + public void testTableSourceProjection() { + TableEnvironment tEnv = createBatchTableEnv(); + + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + hbaseTable.setRowKey(ROW_KEY, Integer.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW<col1 INT>," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT " + + " h.family1.col1, " + + " h.family3.col1, " + + " h.family3.col2, " + + " h.family3.col3 " + + "FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "10,1.01,false,Welt-1\n" + + "20,2.02,true,Welt-2\n" + + "30,3.03,false,Welt-3\n" + + "40,4.04,true,Welt-4\n" + + "50,5.05,false,Welt-5\n" + + "60,6.06,true,Welt-6\n" + + "70,7.07,false,Welt-7\n" + + "80,8.08,true,Welt-8\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test + public void testTableSourceFieldOrder() { + TableEnvironment tEnv = createBatchTableEnv(); + + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + // shuffle order of column registration + hbaseTable.setRowKey(ROW_KEY, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " rowkey INT PRIMARY KEY," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " family1 ROW<col1 INT>" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "1,Hello-1,100,1.01,false,Welt-1,10\n" + + "2,Hello-2,200,2.02,true,Welt-2,20\n" + + "3,Hello-3,300,3.03,false,Welt-3,30\n" + + "4,null,400,4.04,true,Welt-4,40\n" + + "5,Hello-5,500,5.05,false,Welt-5,50\n" + + "6,Hello-6,600,6.06,true,Welt-6,60\n" + + "7,Hello-7,700,7.07,false,Welt-7,70\n" + + "8,null,800,8.08,true,Welt-8,80\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test Review comment: Could we add `testTableSourceWithTableAPI` here when rebase FLINK-18604? ########## File path: flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java ########## @@ -0,0 +1,648 @@ +/* + * 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; + +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.connector.hbase.util.HBaseTableSchema; +import org.apache.flink.connector.hbase.util.PlannerType; +import org.apache.flink.connector.hbase2.source.AbstractTableInputFormat; +import org.apache.flink.connector.hbase2.source.HBaseInputFormat; +import org.apache.flink.connector.hbase2.source.HBaseRowDataInputFormat; +import org.apache.flink.connector.hbase2.source.HBaseRowInputFormat; +import org.apache.flink.connector.hbase2.source.HBaseTableSource; +import org.apache.flink.connector.hbase2.util.HBaseTestBase; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.BatchTableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentInternal; +import org.apache.flink.table.descriptors.HBase; +import org.apache.flink.table.descriptors.Schema; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.hbase.util.PlannerType.OLD_PLANNER; +import static org.apache.flink.table.api.Expressions.$; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +/** + * IT cases for HBase connector (including HBaseTableSource and HBaseTableSink). + */ +@RunWith(Parameterized.class) +public class HBaseConnectorITCase extends HBaseTestBase { + + @Parameterized.Parameter + public PlannerType planner; + + @Parameterized.Parameter(1) + public boolean isLegacyConnector; + + @Override + protected PlannerType planner() { + return planner; + } + + @Parameterized.Parameters(name = "planner = {0}, legacy = {1}") + public static Object[] parameters() { + return new Object[][]{ + new Object[]{PlannerType.BLINK_PLANNER, true}, + new Object[]{PlannerType.BLINK_PLANNER, false}, + new Object[]{PlannerType.OLD_PLANNER, true} + }; + } + + // ------------------------------------------------------------------------------------- + // HBaseTableSource tests + // ------------------------------------------------------------------------------------- + + @Test + public void testTableSourceFullScan() { + TableEnvironment tEnv = createBatchTableEnv(); + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + hbaseTable.setRowKey(ROW_KEY, Integer.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW<col1 INT>," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT " + + " h.family1.col1, " + + " h.family2.col1, " + + " h.family2.col2, " + + " h.family3.col1, " + + " h.family3.col2, " + + " h.family3.col3 " + + "FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "10,Hello-1,100,1.01,false,Welt-1\n" + + "20,Hello-2,200,2.02,true,Welt-2\n" + + "30,Hello-3,300,3.03,false,Welt-3\n" + + "40,null,400,4.04,true,Welt-4\n" + + "50,Hello-5,500,5.05,false,Welt-5\n" + + "60,Hello-6,600,6.06,true,Welt-6\n" + + "70,Hello-7,700,7.07,false,Welt-7\n" + + "80,null,800,8.08,true,Welt-8\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test + public void testTableSourceProjection() { + TableEnvironment tEnv = createBatchTableEnv(); + + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + hbaseTable.setRowKey(ROW_KEY, Integer.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW<col1 INT>," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT " + + " h.family1.col1, " + + " h.family3.col1, " + + " h.family3.col2, " + + " h.family3.col3 " + + "FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "10,1.01,false,Welt-1\n" + + "20,2.02,true,Welt-2\n" + + "30,3.03,false,Welt-3\n" + + "40,4.04,true,Welt-4\n" + + "50,5.05,false,Welt-5\n" + + "60,6.06,true,Welt-6\n" + + "70,7.07,false,Welt-7\n" + + "80,8.08,true,Welt-8\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test + public void testTableSourceFieldOrder() { + TableEnvironment tEnv = createBatchTableEnv(); + + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + // shuffle order of column registration + hbaseTable.setRowKey(ROW_KEY, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " rowkey INT PRIMARY KEY," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " family1 ROW<col1 INT>" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "1,Hello-1,100,1.01,false,Welt-1,10\n" + + "2,Hello-2,200,2.02,true,Welt-2,20\n" + + "3,Hello-3,300,3.03,false,Welt-3,30\n" + + "4,null,400,4.04,true,Welt-4,40\n" + + "5,Hello-5,500,5.05,false,Welt-5,50\n" + + "6,Hello-6,600,6.06,true,Welt-6,60\n" + + "7,Hello-7,700,7.07,false,Welt-7,70\n" + + "8,null,800,8.08,true,Welt-8,80\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + public void testTableSourceWithTableAPI() throws Exception { + StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); + tEnv.connect(new HBase() + .version("1.4.3") Review comment: ```suggestion .version("2.2.3") ``` ########## File path: flink-connectors/pom.xml ########## @@ -43,7 +43,9 @@ under the License. <module>flink-connector-elasticsearch5</module> <module>flink-connector-elasticsearch6</module> <module>flink-connector-elasticsearch7</module> - <module>flink-connector-hbase</module> + <module>flink-connector-hbase-base</module> + <module>flink-connector-hbase-1.4</module> + <module>flink-connector-hbase-2.2</module> Review comment: please also update sql connector module name here : https://github.com/apache/flink/pull/13128/files#diff-87284b7dd728ae85a73c904513781f49R98 ########## File path: flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java ########## @@ -0,0 +1,648 @@ +/* + * 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; + +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.connector.hbase.util.HBaseTableSchema; +import org.apache.flink.connector.hbase.util.PlannerType; +import org.apache.flink.connector.hbase2.source.AbstractTableInputFormat; +import org.apache.flink.connector.hbase2.source.HBaseInputFormat; +import org.apache.flink.connector.hbase2.source.HBaseRowDataInputFormat; +import org.apache.flink.connector.hbase2.source.HBaseRowInputFormat; +import org.apache.flink.connector.hbase2.source.HBaseTableSource; +import org.apache.flink.connector.hbase2.util.HBaseTestBase; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.BatchTableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentInternal; +import org.apache.flink.table.descriptors.HBase; +import org.apache.flink.table.descriptors.Schema; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.hbase.util.PlannerType.OLD_PLANNER; +import static org.apache.flink.table.api.Expressions.$; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +/** + * IT cases for HBase connector (including HBaseTableSource and HBaseTableSink). + */ +@RunWith(Parameterized.class) +public class HBaseConnectorITCase extends HBaseTestBase { + + @Parameterized.Parameter + public PlannerType planner; + + @Parameterized.Parameter(1) + public boolean isLegacyConnector; + + @Override + protected PlannerType planner() { + return planner; + } + + @Parameterized.Parameters(name = "planner = {0}, legacy = {1}") + public static Object[] parameters() { + return new Object[][]{ + new Object[]{PlannerType.BLINK_PLANNER, true}, + new Object[]{PlannerType.BLINK_PLANNER, false}, + new Object[]{PlannerType.OLD_PLANNER, true} + }; + } + + // ------------------------------------------------------------------------------------- + // HBaseTableSource tests + // ------------------------------------------------------------------------------------- + + @Test + public void testTableSourceFullScan() { + TableEnvironment tEnv = createBatchTableEnv(); + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + hbaseTable.setRowKey(ROW_KEY, Integer.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW<col1 INT>," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT " + + " h.family1.col1, " + + " h.family2.col1, " + + " h.family2.col2, " + + " h.family3.col1, " + + " h.family3.col2, " + + " h.family3.col3 " + + "FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "10,Hello-1,100,1.01,false,Welt-1\n" + + "20,Hello-2,200,2.02,true,Welt-2\n" + + "30,Hello-3,300,3.03,false,Welt-3\n" + + "40,null,400,4.04,true,Welt-4\n" + + "50,Hello-5,500,5.05,false,Welt-5\n" + + "60,Hello-6,600,6.06,true,Welt-6\n" + + "70,Hello-7,700,7.07,false,Welt-7\n" + + "80,null,800,8.08,true,Welt-8\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test + public void testTableSourceProjection() { + TableEnvironment tEnv = createBatchTableEnv(); + + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + hbaseTable.setRowKey(ROW_KEY, Integer.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW<col1 INT>," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT " + + " h.family1.col1, " + + " h.family3.col1, " + + " h.family3.col2, " + + " h.family3.col3 " + + "FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "10,1.01,false,Welt-1\n" + + "20,2.02,true,Welt-2\n" + + "30,3.03,false,Welt-3\n" + + "40,4.04,true,Welt-4\n" + + "50,5.05,false,Welt-5\n" + + "60,6.06,true,Welt-6\n" + + "70,7.07,false,Welt-7\n" + + "80,8.08,true,Welt-8\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + @Test + public void testTableSourceFieldOrder() { + TableEnvironment tEnv = createBatchTableEnv(); + + if (isLegacyConnector) { + HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); + // shuffle order of column registration + hbaseTable.setRowKey(ROW_KEY, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL1, String.class); + hbaseTable.addColumn(FAMILY3, F3COL1, Double.class); + hbaseTable.addColumn(FAMILY1, F1COL1, Integer.class); + hbaseTable.addColumn(FAMILY2, F2COL2, Long.class); + hbaseTable.addColumn(FAMILY3, F3COL2, Boolean.class); + hbaseTable.addColumn(FAMILY3, F3COL3, String.class); + ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); + } else { + tEnv.executeSql( + "CREATE TABLE hTable (" + + " rowkey INT PRIMARY KEY," + + " family2 ROW<col1 STRING, col2 BIGINT>," + + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>," + + " family1 ROW<col1 INT>" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + TEST_TABLE_1 + "'," + + " 'zookeeper.quorum' = '" + getZookeeperQuorum() + "'" + + ")"); + } + + Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h"); + + List<Row> results = CollectionUtil.iteratorToList(table.execute().collect()); + String expected = + "1,Hello-1,100,1.01,false,Welt-1,10\n" + + "2,Hello-2,200,2.02,true,Welt-2,20\n" + + "3,Hello-3,300,3.03,false,Welt-3,30\n" + + "4,null,400,4.04,true,Welt-4,40\n" + + "5,Hello-5,500,5.05,false,Welt-5,50\n" + + "6,Hello-6,600,6.06,true,Welt-6,60\n" + + "7,Hello-7,700,7.07,false,Welt-7,70\n" + + "8,null,800,8.08,true,Welt-8,80\n"; + + TestBaseUtils.compareResultAsText(results, expected); + } + + public void testTableSourceWithTableAPI() throws Exception { + StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); + tEnv.connect(new HBase() + .version("1.4.3") Review comment: ```suggestion .version("2.2.3") ``` ########## File path: flink-connectors/pom.xml ########## @@ -43,7 +43,9 @@ under the License. <module>flink-connector-elasticsearch5</module> <module>flink-connector-elasticsearch6</module> <module>flink-connector-elasticsearch7</module> - <module>flink-connector-hbase</module> + <module>flink-connector-hbase-base</module> + <module>flink-connector-hbase-1.4</module> + <module>flink-connector-hbase-2.2</module> Review comment: please also update sql connector module name here : https://github.com/apache/flink/pull/13128/files#diff-87284b7dd728ae85a73c904513781f49R98 ---------------------------------------------------------------- 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: us...@infra.apache.org