From ddd15555e0ca04d5c5cfe6e9d21169a0e685bb92 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 11 May 2021 17:29:15 +0200 Subject: [PATCH 1/2] [FLINK-22623][hbase] Drop BatchTableSource/Sink HBaseTableSource/Sink and related classes This removes the HBaseTableSource/Sink and related classes including various HBaseInputFormats and HBaseSinkFunction. It is possible to read via Table & SQL API and convert the Table to DataStream API (or vice versa) if necessary. DataSet API is not supported anymore. This closes #15905. --- .../flink-connector-hbase-1.4/pom.xml | 53 +-- .../flink/addons/hbase1/TableInputFormat.java | 37 -- .../connector/hbase1/HBase1TableFactory.java | 228 ---------- .../connector/hbase1/HBaseValidator.java | 47 -- .../hbase1/sink/HBaseUpsertTableSink.java | 140 ------ .../hbase1/source/HBaseInputFormat.java | 99 ----- .../hbase1/source/HBaseRowInputFormat.java | 114 ----- .../hbase1/source/HBaseTableSource.java | 83 ---- ....apache.flink.table.factories.TableFactory | 16 - .../hbase1/HBaseConnectorITCase.java | 401 ++++------------- .../connector/hbase1/HBaseDescriptorTest.java | 182 -------- .../hbase1/HBaseTableFactoryTest.java | 214 ---------- .../hbase1/example/HBaseWriteExample.java | 215 ---------- .../example/HBaseWriteStreamExample.java | 105 ----- .../connector/hbase1/util/HBaseTestBase.java | 15 +- .../flink-connector-hbase-2.2/README.md | 89 ---- .../flink-connector-hbase-2.2/pom.xml | 63 +-- .../connector/hbase2/HBase2TableFactory.java | 226 ---------- .../connector/hbase2/HBaseValidator.java | 46 -- .../hbase2/sink/HBaseUpsertTableSink.java | 140 ------ .../hbase2/source/HBaseInputFormat.java | 107 ----- .../hbase2/source/HBaseRowInputFormat.java | 117 ----- .../hbase2/source/HBaseTableSource.java | 83 ---- ....apache.flink.table.factories.TableFactory | 16 - .../hbase2/HBaseConnectorITCase.java | 402 ++++-------------- .../connector/hbase2/HBaseDescriptorTest.java | 152 ------- .../hbase2/HBaseTableFactoryTest.java | 214 ---------- .../hbase2/example/HBaseWriteExample.java | 215 ---------- .../HBaseRowDataAsyncLookupFunctionTest.java | 7 - .../connector/hbase2/util/HBaseTestBase.java | 15 +- .../flink-connector-hbase-base/pom.xml | 32 +- .../hbase/sink/LegacyMutationConverter.java | 56 --- .../source/AbstractHBaseTableSource.java | 194 --------- .../hbase/source/HBaseLookupFunction.java | 158 ------- .../hbase/util/HBaseReadWriteHelper.java | 225 ---------- .../descriptors/AbstractHBaseValidator.java | 75 ---- .../apache/flink/table/descriptors/HBase.java | 173 -------- .../example/HBaseFlinkTestConstants.java | 30 -- .../connector/hbase/util/PlannerType.java | 25 -- flink-python/pyflink/table/descriptors.py | 151 ------- .../pyflink/table/tests/test_descriptor.py | 110 +---- 41 files changed, 223 insertions(+), 4847 deletions(-) delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/addons/hbase1/TableInputFormat.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBase1TableFactory.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBaseValidator.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/sink/HBaseUpsertTableSink.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseInputFormat.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseRowInputFormat.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseTableSource.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseDescriptorTest.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseTableFactoryTest.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/example/HBaseWriteExample.java delete mode 100644 flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/example/HBaseWriteStreamExample.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/README.md delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBase2TableFactory.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBaseValidator.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/sink/HBaseUpsertTableSink.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseInputFormat.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowInputFormat.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseTableSource.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseDescriptorTest.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseTableFactoryTest.java delete mode 100644 flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/example/HBaseWriteExample.java delete mode 100644 flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/LegacyMutationConverter.java delete mode 100644 flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/AbstractHBaseTableSource.java delete mode 100644 flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/HBaseLookupFunction.java delete mode 100644 flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseReadWriteHelper.java delete mode 100644 flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/AbstractHBaseValidator.java delete mode 100644 flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/HBase.java delete mode 100644 flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/example/HBaseFlinkTestConstants.java delete mode 100644 flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/PlannerType.java diff --git a/flink-connectors/flink-connector-hbase-1.4/pom.xml b/flink-connectors/flink-connector-hbase-1.4/pom.xml index f4620482a5469..7287aa3ad2906 100644 --- a/flink-connectors/flink-connector-hbase-1.4/pom.xml +++ b/flink-connectors/flink-connector-hbase-1.4/pom.xml @@ -54,11 +54,7 @@ under the License. - - org.apache.flink - flink-connector-hbase-base_${scala.binary.version} - ${project.version} - + org.apache.flink @@ -67,34 +63,23 @@ under the License. provided - - org.apache.flink - flink-java - ${project.version} - provided - - - - - - org.apache.flink - flink-scala_${scala.binary.version} + flink-streaming-java_${scala.binary.version} ${project.version} provided - + org.apache.flink - flink-streaming-scala_${scala.binary.version} + flink-connector-hbase-base_${scala.binary.version} ${project.version} - provided + org.apache.flink @@ -104,14 +89,7 @@ under the License. true - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${project.version} - provided - - - + org.apache.flink @@ -121,7 +99,6 @@ under the License. test - org.apache.flink flink-hadoop-compatibility_${scala.binary.version} @@ -134,12 +111,14 @@ under the License. + org.apache.flink flink-test-utils_${scala.binary.version} ${project.version} test + org.apache.flink flink-table-common @@ -147,25 +126,14 @@ under the License. test-jar test - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - test - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - test-jar - test - + org.apache.flink flink-table-planner-blink_${scala.binary.version} ${project.version} test + org.apache.flink flink-table-planner-blink_${scala.binary.version} @@ -199,6 +167,7 @@ under the License. + org.apache.hbase hbase-server diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/addons/hbase1/TableInputFormat.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/addons/hbase1/TableInputFormat.java deleted file mode 100644 index 6a8f233edbb89..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/addons/hbase1/TableInputFormat.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.addons.hbase1; - -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.connector.hbase1.source.HBaseInputFormat; - -/** - * {@link InputFormat} subclass that wraps the access for HTables. - * - * @deprecated please use {@link org.apache.flink.connector.hbase1.source.HBaseInputFormat}. - */ -@Deprecated -public abstract class TableInputFormat extends HBaseInputFormat { - private static final long serialVersionUID = 1L; - - public TableInputFormat(org.apache.hadoop.conf.Configuration hConf) { - super(hConf); - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBase1TableFactory.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBase1TableFactory.java deleted file mode 100644 index aa8abc5a1d08d..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBase1TableFactory.java +++ /dev/null @@ -1,228 +0,0 @@ -/* - * 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.hbase1; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.connector.hbase.options.HBaseWriteOptions; -import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.connector.hbase1.sink.HBaseUpsertTableSink; -import org.apache.flink.connector.hbase1.source.HBaseTableSource; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.factories.StreamTableSinkFactory; -import org.apache.flink.table.factories.StreamTableSourceFactory; -import org.apache.flink.table.sinks.StreamTableSink; -import org.apache.flink.table.sources.StreamTableSource; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.types.Row; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; - -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_PROPERTIES; -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_TABLE_NAME; -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_TYPE_VALUE_HBASE; -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_VERSION_VALUE_143; -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL; -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS; -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE; -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_ZK_NODE_PARENT; -import static org.apache.flink.connector.hbase1.HBaseValidator.CONNECTOR_ZK_QUORUM; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_DATA_TYPE; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE; - -/** Factory for creating configured instances of {@link HBaseTableSource} or sink. */ -@Internal -public class HBase1TableFactory - implements StreamTableSourceFactory, StreamTableSinkFactory> { - - @Override - public StreamTableSource createStreamTableSource(Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - // create default configuration from current runtime env (`hbase-site.xml` in classpath) - // first, - Configuration hbaseClientConf = getHBaseConf(descriptorProperties); - - String hTableName = descriptorProperties.getString(CONNECTOR_TABLE_NAME); - TableSchema tableSchema = - TableSchemaUtils.getPhysicalSchema(descriptorProperties.getTableSchema(SCHEMA)); - HBaseTableSchema hbaseSchema = validateTableSchema(tableSchema); - return new HBaseTableSource(hbaseClientConf, hTableName, hbaseSchema, null); - } - - @Override - public StreamTableSink> createStreamTableSink( - Map properties) { - - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - - TableSchema tableSchema = - TableSchemaUtils.getPhysicalSchema(descriptorProperties.getTableSchema(SCHEMA)); - HBaseTableSchema hbaseSchema = validateTableSchema(tableSchema); - - Configuration hbaseClientConf = getHBaseConf(descriptorProperties); - - HBaseWriteOptions.Builder writeBuilder = HBaseWriteOptions.builder(); - descriptorProperties - .getOptionalInt(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS) - .ifPresent(writeBuilder::setBufferFlushMaxRows); - descriptorProperties - .getOptionalMemorySize(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE) - .ifPresent(v -> writeBuilder.setBufferFlushMaxSizeInBytes(v.getBytes())); - descriptorProperties - .getOptionalDuration(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL) - .ifPresent(v -> writeBuilder.setBufferFlushIntervalMillis(v.toMillis())); - - return new HBaseUpsertTableSink( - descriptorProperties.getString(CONNECTOR_TABLE_NAME), - hbaseSchema, - hbaseClientConf, - writeBuilder.build()); - } - - private HBaseTableSchema validateTableSchema(TableSchema schema) { - HBaseTableSchema hbaseSchema = new HBaseTableSchema(); - String[] fieldNames = schema.getFieldNames(); - TypeInformation[] fieldTypes = schema.getFieldTypes(); - for (int i = 0; i < fieldNames.length; i++) { - String name = fieldNames[i]; - TypeInformation type = fieldTypes[i]; - if (type instanceof RowTypeInfo) { - RowTypeInfo familyType = (RowTypeInfo) type; - String[] qualifierNames = familyType.getFieldNames(); - TypeInformation[] qualifierTypes = familyType.getFieldTypes(); - for (int j = 0; j < familyType.getArity(); j++) { - // HBase connector doesn't support LocalDateTime - // use Timestamp as conversion class for now. - Class clazz = qualifierTypes[j].getTypeClass(); - if (LocalDateTime.class.equals(clazz)) { - clazz = Timestamp.class; - } else if (LocalDate.class.equals(clazz)) { - clazz = Date.class; - } else if (LocalTime.class.equals(clazz)) { - clazz = Time.class; - } - hbaseSchema.addColumn(name, qualifierNames[j], clazz); - } - } else { - hbaseSchema.setRowKey(name, type.getTypeClass()); - } - } - return hbaseSchema; - } - - private DescriptorProperties getValidatedProperties(Map properties) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(true); - descriptorProperties.putProperties(properties); - new HBaseValidator().validate(descriptorProperties); - return descriptorProperties; - } - - @Override - public Map requiredContext() { - Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_HBASE); // hbase - context.put(CONNECTOR_VERSION, hbaseVersion()); // version - context.put(CONNECTOR_PROPERTY_VERSION, "1"); // backwards compatibility - return context; - } - - @Override - public List supportedProperties() { - List properties = new ArrayList<>(); - - properties.add(CONNECTOR_TABLE_NAME); - properties.add(CONNECTOR_ZK_QUORUM); - properties.add(CONNECTOR_ZK_NODE_PARENT); - properties.add(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE); - properties.add(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS); - properties.add(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL); - - // schema - properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); - properties.add(SCHEMA + ".#." + SCHEMA_TYPE); - properties.add(SCHEMA + ".#." + SCHEMA_NAME); - // computed column - properties.add(SCHEMA + ".#." + EXPR); - - // watermark - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_ROWTIME); - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_EXPR); - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_DATA_TYPE); - - // table constraint - properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_NAME); - properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_COLUMNS); - - // comment - properties.add(COMMENT); - - // HBase properties - properties.add(CONNECTOR_PROPERTIES + ".*"); - - return properties; - } - - private String hbaseVersion() { - return CONNECTOR_VERSION_VALUE_143; - } - - private static Configuration getHBaseConf(DescriptorProperties descriptorProperties) { - Configuration hbaseClientConf = HBaseConfigurationUtil.createHBaseConf(); - descriptorProperties - .getOptionalString(CONNECTOR_ZK_QUORUM) - .ifPresent(zkQ -> hbaseClientConf.set(HConstants.ZOOKEEPER_QUORUM, zkQ)); - - descriptorProperties - .getOptionalString(CONNECTOR_ZK_NODE_PARENT) - .ifPresent(v -> hbaseClientConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, v)); - - // add HBase properties - descriptorProperties - .getPropertiesWithPrefix(CONNECTOR_PROPERTIES) - .forEach(hbaseClientConf::set); - return hbaseClientConf; - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBaseValidator.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBaseValidator.java deleted file mode 100644 index ea27aaf56d7a6..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBaseValidator.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.hbase1; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.descriptors.AbstractHBaseValidator; - -/** - * The validator for HBase. More features to be supported, e.g., batch read/write, async api(support - * from hbase version 2.0.0), Caching for LookupFunction. - */ -@Internal -public class HBaseValidator extends AbstractHBaseValidator { - - public static final String CONNECTOR_VERSION_VALUE_143 = "1.4.3"; - - @Override - protected boolean validateZkQuorum() { - return false; - } - - @Override - protected String getConnectorVersion() { - return CONNECTOR_VERSION_VALUE_143; - } - - @Override - protected boolean zkQuorumIsOptional() { - return false; - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/sink/HBaseUpsertTableSink.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/sink/HBaseUpsertTableSink.java deleted file mode 100644 index 3463f7dffeea0..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/sink/HBaseUpsertTableSink.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.hbase1.sink; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.connector.hbase.options.HBaseWriteOptions; -import org.apache.flink.connector.hbase.sink.HBaseSinkFunction; -import org.apache.flink.connector.hbase.sink.LegacyMutationConverter; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.sinks.UpsertStreamTableSink; -import org.apache.flink.table.utils.TableConnectorUtils; -import org.apache.flink.types.Row; - -import org.apache.hadoop.conf.Configuration; - -import java.util.Arrays; - -import static org.apache.flink.util.Preconditions.checkArgument; - -/** An upsert {@link UpsertStreamTableSink} for HBase. */ -@Internal -public class HBaseUpsertTableSink implements UpsertStreamTableSink { - - private final HBaseTableSchema hbaseTableSchema; - private final TableSchema tableSchema; - private final HBaseWriteOptions writeOptions; - private final Configuration hbaseConf; - private final String tableName; - - public HBaseUpsertTableSink( - String tableName, - HBaseTableSchema hbaseTableSchema, - Configuration hbaseConf, - HBaseWriteOptions writeOptions) { - checkArgument( - hbaseTableSchema.getRowKeyName().isPresent(), - "HBaseUpsertTableSink requires rowkey is set."); - this.hbaseTableSchema = hbaseTableSchema; - this.tableSchema = hbaseTableSchema.convertsToTableSchema(); - this.hbaseConf = hbaseConf; - this.writeOptions = writeOptions; - this.tableName = tableName; - } - - @Override - public void setKeyFields(String[] keys) { - // hbase always upsert on rowkey, ignore query keys. - // Actually, we should verify the query key is the same with rowkey. - // However, the query key extraction doesn't work well in some scenarios - // (e.g. concat key fields will lose key information). So we skip key validation currently. - } - - @Override - public void setIsAppendOnly(Boolean isAppendOnly) { - // hbase always upsert on rowkey, even works in append only mode. - } - - @Override - public TypeInformation getRecordType() { - return tableSchema.toRowType(); - } - - @Override - public TableSchema getTableSchema() { - return tableSchema; - } - - @Override - public DataStreamSink consumeDataStream(DataStream> dataStream) { - HBaseSinkFunction sinkFunction = - new HBaseSinkFunction( - tableName, - hbaseConf, - new LegacyMutationConverter(hbaseTableSchema), - writeOptions.getBufferFlushMaxSizeInBytes(), - writeOptions.getBufferFlushMaxRows(), - writeOptions.getBufferFlushIntervalMillis()); - return dataStream - .addSink(sinkFunction) - .setParallelism(dataStream.getParallelism()) - .name( - TableConnectorUtils.generateRuntimeName( - this.getClass(), tableSchema.getFieldNames())); - } - - @Override - public TableSink> configure( - String[] fieldNames, TypeInformation[] fieldTypes) { - if (!Arrays.equals(getFieldNames(), fieldNames) - || !Arrays.equals(getFieldTypes(), fieldTypes)) { - throw new ValidationException( - "Reconfiguration with different fields is not allowed. " - + "Expected: " - + Arrays.toString(getFieldNames()) - + " / " - + Arrays.toString(getFieldTypes()) - + ". " - + "But was: " - + Arrays.toString(fieldNames) - + " / " - + Arrays.toString(fieldTypes)); - } - - return new HBaseUpsertTableSink(tableName, hbaseTableSchema, hbaseConf, writeOptions); - } - - @VisibleForTesting - public HBaseTableSchema getHBaseTableSchema() { - return hbaseTableSchema; - } - - @VisibleForTesting - public HBaseWriteOptions getWriteOptions() { - return writeOptions; - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseInputFormat.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseInputFormat.java deleted file mode 100644 index 4ef8350a183fa..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseInputFormat.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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.hbase1.source; - -import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.java.tuple.Tuple; - -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; - -/** {@link InputFormat} subclass that wraps the access for HTables. */ -@Experimental -public abstract class HBaseInputFormat extends AbstractTableInputFormat { - - private static final long serialVersionUID = 1L; - - /** - * Constructs a {@link InputFormat} with hbase configuration to read data from hbase. - * - * @param hConf The configuration that connect to hbase. At least hbase.zookeeper.quorum and - * zookeeper.znode.parent need to be set. - */ - public HBaseInputFormat(org.apache.hadoop.conf.Configuration hConf) { - super(hConf); - } - - /** - * Returns an instance of Scan that retrieves the required subset of records from the HBase - * table. - * - * @return The appropriate instance of Scan for this usecase. - */ - @Override - protected abstract Scan getScanner(); - - /** - * What table is to be read. Per instance of a TableInputFormat derivative only a single - * tablename is possible. - * - * @return The name of the table - */ - @Override - protected abstract String getTableName(); - - /** - * The output from HBase is always an instance of {@link Result}. This method is to copy the - * data in the Result instance into the required {@link Tuple} - * - * @param r The Result instance from HBase that needs to be converted - * @return The appropriate instance of {@link Tuple} that contains the needed information. - */ - protected abstract T mapResultToTuple(Result r); - - @Override - protected void initTable() { - if (table == null) { - table = createTable(); - } - if (table != null && scan == null) { - scan = getScanner(); - } - } - - /** Create an {@link HTable} instance and set it into this format. */ - private HTable createTable() { - LOG.info("Initializing HBaseConfiguration"); - org.apache.hadoop.conf.Configuration hConf = getHadoopConfiguration(); - - try { - return new HTable(hConf, getTableName()); - } catch (Exception e) { - LOG.error("Error instantiating a new HTable instance", e); - } - return null; - } - - @Override - protected T mapResultToOutType(Result r) { - return mapResultToTuple(r); - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseRowInputFormat.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseRowInputFormat.java deleted file mode 100644 index 52af8d1ae8f9b..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseRowInputFormat.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.hbase1.source; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.connector.hbase.util.HBaseReadWriteHelper; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.types.Row; - -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; - -/** - * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row} - */ -@Internal -public class HBaseRowInputFormat extends AbstractTableInputFormat - implements ResultTypeQueryable { - - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(HBaseRowInputFormat.class); - - private final String tableName; - private final HBaseTableSchema schema; - - private transient HBaseReadWriteHelper readHelper; - - public HBaseRowInputFormat( - org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) { - super(conf); - this.tableName = tableName; - this.schema = schema; - } - - @Override - public void initTable() throws IOException { - this.readHelper = new HBaseReadWriteHelper(schema); - if (table == null) { - connectToTable(); - } - if (table != null && scan == null) { - scan = getScanner(); - } - } - - @Override - protected Scan getScanner() { - return readHelper.createScan(); - } - - @Override - public String getTableName() { - return tableName; - } - - @Override - protected Row mapResultToOutType(Result res) { - return readHelper.parseToRow(res); - } - - private void connectToTable() throws IOException { - try { - connection = ConnectionFactory.createConnection(getHadoopConfiguration()); - table = (HTable) connection.getTable(TableName.valueOf(tableName)); - } catch (TableNotFoundException tnfe) { - LOG.error("The table " + tableName + " not found ", tnfe); - throw new RuntimeException("HBase table '" + tableName + "' not found.", tnfe); - } - } - - @Override - public TypeInformation getProducedType() { - // split the fieldNames - String[] famNames = schema.getFamilyNames(); - TypeInformation[] typeInfos = new TypeInformation[famNames.length]; - int i = 0; - for (String family : famNames) { - typeInfos[i] = - new RowTypeInfo( - schema.getQualifierTypes(family), schema.getQualifierNames(family)); - i++; - } - return new RowTypeInfo(typeInfos, famNames); - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseTableSource.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseTableSource.java deleted file mode 100644 index d3bc81accfeb7..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseTableSource.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.hbase1.source; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.connector.hbase.source.AbstractHBaseTableSource; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.types.Row; - -import org.apache.hadoop.conf.Configuration; - -/** - * Creates a TableSource to scan an HBase table. - * - *

The table name and required HBase configuration is passed during {@link HBaseTableSource} - * construction. Use {@link #addColumn(String, String, Class)} to specify the family, qualifier, and - * type of columns to scan. - * - *

The TableSource returns {@link Row} with nested Rows for each column family. - * - *

The HBaseTableSource is used as shown in the example below. - * - *

{@code
- * HBaseTableSource hSrc = new HBaseTableSource(conf, "hTable");
- * hSrc.setRowKey("rowkey", String.class);
- * hSrc.addColumn("fam1", "col1", byte[].class);
- * hSrc.addColumn("fam1", "col2", Integer.class);
- * hSrc.addColumn("fam2", "col1", String.class);
- *
- * tableEnv.registerTableSourceInternal("hTable", hSrc);
- * Table res = tableEnv.sqlQuery(
- *   "SELECT t.fam2.col1, SUM(t.fam1.col2) FROM hTable AS t " +
- *   "WHERE t.rowkey LIKE 'flink%' GROUP BY t.fam2.col1");
- * }
- */ -@Internal -public class HBaseTableSource extends AbstractHBaseTableSource { - - /** - * The HBase configuration and the name of the table to read. - * - * @param conf hbase configuration - * @param tableName the tableName - */ - public HBaseTableSource(Configuration conf, String tableName) { - this(conf, tableName, new HBaseTableSchema(), null); - } - - public HBaseTableSource( - Configuration conf, - String tableName, - HBaseTableSchema hbaseSchema, - int[] projectFields) { - super(conf, tableName, hbaseSchema, projectFields); - } - - @Override - public HBaseTableSource projectFields(int[] fields) { - return new HBaseTableSource(conf, tableName, hbaseSchema, fields); - } - - @Override - public InputFormat getInputFormat(HBaseTableSchema projectedSchema) { - return new HBaseRowInputFormat(conf, tableName, projectedSchema); - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-hbase-1.4/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index 5d4e1224ab071..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ /dev/null @@ -1,16 +0,0 @@ -# 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.hbase1.HBase1TableFactory diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseConnectorITCase.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseConnectorITCase.java index fc8a555806cf3..1ad2dd2d19649 100644 --- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseConnectorITCase.java +++ b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseConnectorITCase.java @@ -18,44 +18,26 @@ package org.apache.flink.connector.hbase1; -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.hbase1.source.AbstractTableInputFormat; -import org.apache.flink.connector.hbase1.source.HBaseInputFormat; import org.apache.flink.connector.hbase1.source.HBaseRowDataInputFormat; -import org.apache.flink.connector.hbase1.source.HBaseRowInputFormat; -import org.apache.flink.connector.hbase1.source.HBaseTableSource; import org.apache.flink.connector.hbase1.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.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; @@ -63,70 +45,37 @@ 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) +/** IT cases for HBase connector (source and sink). */ 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," - + " family2 ROW," - + " family3 ROW," - + " rowkey INT," - + " PRIMARY KEY (rowkey) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'hbase-1.4'," - + " 'table-name' = '" - + TEST_TABLE_1 - + "'," - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "'" - + ")"); - } + TableEnvironment tEnv = TableEnvironment.create(batchSettings); + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW," + + " family2 ROW," + + " family3 ROW," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-1.4'," + + " 'table-name' = '" + + TEST_TABLE_1 + + "'," + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "'" + + ")"); Table table = tEnv.sqlQuery( @@ -155,36 +104,24 @@ public void testTableSourceFullScan() { @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," - + " family2 ROW," - + " family3 ROW," - + " rowkey INT," - + " PRIMARY KEY (rowkey) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'hbase-1.4'," - + " 'table-name' = '" - + TEST_TABLE_1 - + "'," - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "'" - + ")"); - } + TableEnvironment tEnv = TableEnvironment.create(batchSettings); + + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW," + + " family2 ROW," + + " family3 ROW," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-1.4'," + + " 'table-name' = '" + + TEST_TABLE_1 + + "'," + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "'" + + ")"); Table table = tEnv.sqlQuery( @@ -211,36 +148,23 @@ public void testTableSourceProjection() { @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," - + " family3 ROW," - + " family1 ROW" - + ") WITH (" - + " 'connector' = 'hbase-1.4'," - + " 'table-name' = '" - + TEST_TABLE_1 - + "'," - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "'" - + ")"); - } + TableEnvironment tEnv = TableEnvironment.create(batchSettings); + + tEnv.executeSql( + "CREATE TABLE hTable (" + + " rowkey INT PRIMARY KEY," + + " family2 ROW," + + " family3 ROW," + + " family1 ROW" + + ") WITH (" + + " 'connector' = 'hbase-1.4'," + + " 'table-name' = '" + + TEST_TABLE_1 + + "'," + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "'" + + ")"); Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h"); @@ -259,74 +183,23 @@ public void testTableSourceFieldOrder() { } @Test - public void testTableSourceWithTableAPI() throws Exception { - StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); - tEnv.connect( - new HBase() - .version("1.4.3") - .tableName(TEST_TABLE_1) - .zookeeperQuorum(getZookeeperQuorum())) - .withSchema( - new Schema() - .field("rowkey", DataTypes.INT()) - .field( - "family2", - DataTypes.ROW( - DataTypes.FIELD("col1", DataTypes.STRING()), - DataTypes.FIELD("col2", DataTypes.BIGINT()))) - .field( - "family3", - DataTypes.ROW( - DataTypes.FIELD("col1", DataTypes.DOUBLE()), - DataTypes.FIELD("col2", DataTypes.BOOLEAN()), - DataTypes.FIELD("col3", DataTypes.STRING()))) - .field( - "family1", - DataTypes.ROW(DataTypes.FIELD("col1", DataTypes.INT())))) - .createTemporaryTable("hTable"); - Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h"); - List results = CollectionUtil.iteratorToList(table.execute().collect()); - String expected = - "+I[1, +I[Hello-1, 100], +I[1.01, false, Welt-1], +I[10]]\n" - + "+I[2, +I[Hello-2, 200], +I[2.02, true, Welt-2], +I[20]]\n" - + "+I[3, +I[Hello-3, 300], +I[3.03, false, Welt-3], +I[30]]\n" - + "+I[4, +I[null, 400], +I[4.04, true, Welt-4], +I[40]]\n" - + "+I[5, +I[Hello-5, 500], +I[5.05, false, Welt-5], +I[50]]\n" - + "+I[6, +I[Hello-6, 600], +I[6.06, true, Welt-6], +I[60]]\n" - + "+I[7, +I[Hello-7, 700], +I[7.07, false, Welt-7], +I[70]]\n" - + "+I[8, +I[null, 800], +I[8.08, true, Welt-8], +I[80]]\n"; + public void testTableSourceReadAsByteArray() { + TableEnvironment tEnv = TableEnvironment.create(batchSettings); - TestBaseUtils.compareResultAsText(results, expected); - } - - @Test - public void testTableSourceReadAsByteArray() throws Exception { - TableEnvironment tEnv = createBatchTableEnv(); - - if (isLegacyConnector) { - // fetch row2 from the table till the end - HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); - hbaseTable.addColumn(FAMILY2, F2COL1, byte[].class); - hbaseTable.addColumn(FAMILY2, F2COL2, byte[].class); - hbaseTable.setRowKey(ROW_KEY, Integer.class); - ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); - } else { - tEnv.executeSql( - "CREATE TABLE hTable (" - + " family2 ROW," - + " rowkey INT" - + // no primary key syntax - ") WITH (" - + " 'connector' = 'hbase-1.4'," - + " 'table-name' = '" - + TEST_TABLE_1 - + "'," - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "'" - + ")"); - } + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family2 ROW," + + " rowkey INT" + + // no primary key syntax + ") WITH (" + + " 'connector' = 'hbase-1.4'," + + " 'table-name' = '" + + TEST_TABLE_1 + + "'," + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "'" + + ")"); tEnv.registerFunction("toUTF8", new ToUTF8()); tEnv.registerFunction("toLong", new ToLong()); @@ -351,22 +224,6 @@ public void testTableSourceReadAsByteArray() throws Exception { TestBaseUtils.compareResultAsText(results, expected); } - @Test - public void testTableInputFormat() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> result = - env.createInput(new InputFormatForTestTable(getConf())) - .reduce( - (ReduceFunction>) - (v1, v2) -> Tuple1.of(v1.f0 + v2.f0)); - - List> resultSet = result.collect(); - - assertEquals(1, resultSet.size()); - assertEquals(360, (int) resultSet.get(0).f0); - } - @Test public void testTableSink() throws Exception { StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -393,7 +250,7 @@ public void testTableSink() throws Exception { tEnv.executeSql(query).await(); // start a batch scan job to verify contents in HBase table - TableEnvironment batchEnv = createBatchTableEnv(); + TableEnvironment batchEnv = TableEnvironment.create(batchSettings); batchEnv.executeSql(table2DDL); Table table = @@ -425,14 +282,6 @@ public void testTableSink() throws Exception { @Test public void testTableSourceSinkWithDDL() throws Exception { - if (OLD_PLANNER.equals(planner) || isLegacyConnector) { - // only test for blink planner and new connector, because types - // TIMESTAMP/DATE/TIME/DECIMAL works well in - // new connector(using blink-planner), but exits some precision problem in old planner - // or legacy connector. - return; - } - StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); @@ -457,7 +306,7 @@ public void testTableSourceSinkWithDDL() throws Exception { tEnv.executeSql(insertStatement).await(); // start a batch scan job to verify contents in HBase table - TableEnvironment batchEnv = createBatchTableEnv(); + TableEnvironment batchEnv = TableEnvironment.create(batchSettings); batchEnv.executeSql(table3DDL); String query = "SELECT " @@ -504,12 +353,6 @@ public void testTableSourceSinkWithDDL() throws Exception { @Test public void testHBaseLookupTableSource() { - if (OLD_PLANNER.equals(planner) || isLegacyConnector) { - // lookup table source is only supported in blink planner, skip for old planner - // types TIMESTAMP/DATE/TIME/DECIMAL works well in new connector, skip legacy connector - return; - } - StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); @@ -581,12 +424,8 @@ public void testHBaseLookupTableSource() { public void testTableInputFormatOpenClose() throws IOException { HBaseTableSchema tableSchema = new HBaseTableSchema(); tableSchema.addColumn(FAMILY1, F1COL1, byte[].class); - AbstractTableInputFormat inputFormat; - if (isLegacyConnector) { - inputFormat = new HBaseRowInputFormat(getConf(), TEST_TABLE_1, tableSchema); - } else { - inputFormat = new HBaseRowDataInputFormat(getConf(), TEST_TABLE_1, tableSchema, "null"); - } + AbstractTableInputFormat inputFormat = + new HBaseRowDataInputFormat(getConf(), TEST_TABLE_1, tableSchema, "null"); inputFormat.open(inputFormat.createInputSplits(1)[0]); assertNotNull(inputFormat.getConnection()); assertNotNull(inputFormat.getConnection().getTable(TableName.valueOf(TEST_TABLE_1))); @@ -615,16 +454,6 @@ public void testTableInputFormatOpenClose() throws IOException { // ------------------------------- Utilities ------------------------------------------------- - /** Creates a Batch {@link TableEnvironment} depends on the {@link #planner} context. */ - private TableEnvironment createBatchTableEnv() { - if (OLD_PLANNER.equals(planner)) { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - return BatchTableEnvironment.create(env, new TableConfig()); - } else { - return TableEnvironment.create(batchSettings); - } - } - /** A {@link ScalarFunction} that maps byte arrays to UTF-8 strings. */ public static class ToUTF8 extends ScalarFunction { private static final long serialVersionUID = 1L; @@ -643,31 +472,6 @@ public long eval(byte[] bytes) { } } - /** A {@link HBaseInputFormat} for testing. */ - public static class InputFormatForTestTable extends HBaseInputFormat> { - private static final long serialVersionUID = 1L; - - public InputFormatForTestTable(org.apache.hadoop.conf.Configuration hConf) { - super(hConf); - } - - @Override - protected Scan getScanner() { - return new Scan(); - } - - @Override - protected String getTableName() { - return TEST_TABLE_1; - } - - @Override - protected Tuple1 mapResultToTuple(Result r) { - return new Tuple1<>( - Bytes.toInt(r.getValue(Bytes.toBytes(FAMILY1), Bytes.toBytes(F1COL1)))); - } - } - private String createHBaseTableDDL(String tableName, boolean testTimeAndDecimalTypes) { StringBuilder family4Statement = new StringBuilder(); if (testTimeAndDecimalTypes) { @@ -677,45 +481,24 @@ private String createHBaseTableDDL(String tableName, boolean testTimeAndDecimalT family4Statement.append(", col4 DECIMAL(12, 4)"); family4Statement.append("> \n"); } - if (isLegacyConnector) { - return "CREATE TABLE " - + tableName - + "(\n" - + " rowkey INT,\n" - + " family1 ROW,\n" - + " family2 ROW,\n" - + " family3 ROW" - + family4Statement.toString() - + ") WITH (\n" - + " 'connector.type' = 'hbase',\n" - + " 'connector.version' = '1.4.3',\n" - + " 'connector.table-name' = '" - + tableName - + "',\n" - + " 'connector.zookeeper.quorum' = '" - + getZookeeperQuorum() - + "',\n" - + " 'connector.zookeeper.znode.parent' = '/hbase' " - + ")"; - } else { - return "CREATE TABLE " - + tableName - + "(\n" - + " rowkey INT," - + " family1 ROW,\n" - + " family2 ROW,\n" - + " family3 ROW" - + family4Statement.toString() - + ") WITH (\n" - + " 'connector' = 'hbase-1.4',\n" - + " 'table-name' = '" - + tableName - + "',\n" - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "',\n" - + " 'zookeeper.znode.parent' = '/hbase' " - + ")"; - } + + return "CREATE TABLE " + + tableName + + "(\n" + + " rowkey INT," + + " family1 ROW,\n" + + " family2 ROW,\n" + + " family3 ROW" + + family4Statement.toString() + + ") WITH (\n" + + " 'connector' = 'hbase-1.4',\n" + + " 'table-name' = '" + + tableName + + "',\n" + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "',\n" + + " 'zookeeper.znode.parent' = '/hbase' " + + ")"; } } diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseDescriptorTest.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseDescriptorTest.java deleted file mode 100644 index 3bb014bde4c2e..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseDescriptorTest.java +++ /dev/null @@ -1,182 +0,0 @@ -/* - * 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.hbase1; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.internal.Registration; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.descriptors.ConnectTableDescriptor; -import org.apache.flink.table.descriptors.Descriptor; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.DescriptorTestBase; -import org.apache.flink.table.descriptors.DescriptorValidator; -import org.apache.flink.table.descriptors.FormatDescriptor; -import org.apache.flink.table.descriptors.HBase; -import org.apache.flink.table.descriptors.Rowtime; -import org.apache.flink.table.descriptors.Schema; -import org.apache.flink.table.descriptors.StreamTableDescriptor; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; - -/** Test case for {@link HBase} descriptor. */ -public class HBaseDescriptorTest extends DescriptorTestBase { - - @Override - protected List descriptors() { - HBase hbaseDesc0 = - new HBase() - .version("1.4.3") - .tableName("testNs:table0") - .zookeeperQuorum("localhost:2181,localhost:2182,localhost:2183") - .zookeeperNodeParent("/hbase/root-dir"); - - HBase hbaseDesc1 = - new HBase() - .version("1.4.3") - .tableName("testNs:table1") - .zookeeperQuorum("localhost:2181") - .zookeeperNodeParent("/hbase/root") - .writeBufferFlushInterval("2s") - .writeBufferFlushMaxRows(100) - .writeBufferFlushMaxSize("1mb"); - - HBase hbaseDesc2 = - new HBase() - .version("1.4.3") - .tableName("testNs:table1") - .zookeeperQuorum("localhost:2181") - .zookeeperNodeParent("/hbase/root") - .writeBufferFlushInterval("2s") - .writeBufferFlushMaxRows(100) - .writeBufferFlushMaxSize("1mb") - .property("hbase.security.authentication", "kerberos"); - - return Arrays.asList(hbaseDesc0, hbaseDesc1, hbaseDesc2); - } - - @Override - protected List> properties() { - Map prop0 = new HashMap<>(); - prop0.put("connector.version", "1.4.3"); - prop0.put("connector.type", "hbase"); - prop0.put("connector.table-name", "testNs:table0"); - prop0.put("connector.zookeeper.quorum", "localhost:2181,localhost:2182,localhost:2183"); - prop0.put("connector.zookeeper.znode.parent", "/hbase/root-dir"); - prop0.put("connector.property-version", "1"); - - Map prop1 = new HashMap<>(); - prop1.put("connector.version", "1.4.3"); - prop1.put("connector.type", "hbase"); - prop1.put("connector.table-name", "testNs:table1"); - prop1.put("connector.zookeeper.quorum", "localhost:2181"); - prop1.put("connector.zookeeper.znode.parent", "/hbase/root"); - prop1.put("connector.property-version", "1"); - prop1.put("connector.write.buffer-flush.interval", "2s"); - prop1.put("connector.write.buffer-flush.max-rows", "100"); - prop1.put("connector.write.buffer-flush.max-size", "1 mb"); - - Map prop2 = new HashMap<>(); - prop2.put("connector.version", "1.4.3"); - prop2.put("connector.type", "hbase"); - prop2.put("connector.table-name", "testNs:table1"); - prop2.put("connector.zookeeper.quorum", "localhost:2181"); - prop2.put("connector.zookeeper.znode.parent", "/hbase/root"); - prop2.put("connector.property-version", "1"); - prop2.put("connector.write.buffer-flush.interval", "2s"); - prop2.put("connector.write.buffer-flush.max-rows", "100"); - prop2.put("connector.write.buffer-flush.max-size", "1 mb"); - prop2.put("connector.properties.hbase.security.authentication", "kerberos"); - - return Arrays.asList(prop0, prop1, prop2); - } - - @Override - protected DescriptorValidator validator() { - return new HBaseValidator(); - } - - @Test - public void testRequiredFields() { - HBase hbaseDesc0 = new HBase(); - HBase hbaseDesc1 = - new HBase() - .version("1.4.3") - .zookeeperQuorum("localhost:2181") - .zookeeperNodeParent("/hbase/root"); // no table name - HBase hbaseDesc2 = - new HBase() - .version("1.4.3") - .tableName("ns:table") - .zookeeperNodeParent("/hbase/root"); // no zookeeper quorum - HBase hbaseDesc3 = - new HBase().tableName("ns:table").zookeeperQuorum("localhost:2181"); // no version - - HBase[] testCases = new HBase[] {hbaseDesc0, hbaseDesc1, hbaseDesc2, hbaseDesc3}; - for (int i = 0; i < testCases.length; i++) { - HBase hbaseDesc = testCases[i]; - DescriptorProperties properties = new DescriptorProperties(); - properties.putProperties(hbaseDesc.toProperties()); - boolean caughtExpectedException = false; - try { - validator().validate(properties); - } catch (ValidationException e) { - caughtExpectedException = true; - } - Assert.assertTrue( - "The case#" + i + " didn't get the expected error", caughtExpectedException); - } - } - - @Test - public void testFormatNeed() { - String expected = - "The connector org.apache.flink.table.descriptors.HBase does not require a format description but org.apache.flink.connector.hbase1.HBaseDescriptorTest$1 found."; - AtomicReference reference = new AtomicReference<>(); - HBase hBase = new HBase(); - Registration registration = (path, table) -> reference.set((CatalogTableImpl) table); - ConnectTableDescriptor descriptor = - new StreamTableDescriptor(registration, hBase) - .withFormat( - new FormatDescriptor("myFormat", 1) { - @Override - protected Map toFormatProperties() { - return new HashMap<>(); - } - }) - .withSchema( - new Schema() - .field("f0", DataTypes.INT()) - .rowtime(new Rowtime().timestampsFromField("f0"))); - String actual = null; - try { - descriptor.toProperties(); - } catch (Exception e) { - actual = e.getMessage(); - } - Assert.assertEquals(expected, actual); - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseTableFactoryTest.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseTableFactoryTest.java deleted file mode 100644 index 0463452874234..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseTableFactoryTest.java +++ /dev/null @@ -1,214 +0,0 @@ -/* - * Copyright The Apache Software Foundation - * - * 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.hbase1; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.connector.hbase.options.HBaseWriteOptions; -import org.apache.flink.connector.hbase.source.HBaseLookupFunction; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.connector.hbase1.sink.HBaseUpsertTableSink; -import org.apache.flink.connector.hbase1.source.HBaseTableSource; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.factories.TableFactoryService; -import org.apache.flink.table.functions.TableFunction; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.sources.TableSource; -import org.apache.flink.types.Row; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; - -import static org.apache.flink.table.descriptors.Schema.SCHEMA; - -/** UT for HBaseTableFactory. */ -public class HBaseTableFactoryTest { - private static final String FAMILY1 = "f1"; - private static final String FAMILY2 = "f2"; - private static final String FAMILY3 = "f3"; - private static final String FAMILY4 = "f4"; - private static final String COL1 = "c1"; - private static final String COL2 = "c2"; - private static final String COL3 = "c3"; - private static final String COL4 = "c4"; - private static final String ROWKEY = "rowkey"; - - private DescriptorProperties createDescriptor(TableSchema tableSchema) { - Map tableProperties = new HashMap<>(); - tableProperties.put("connector.type", "hbase"); - tableProperties.put("connector.version", "1.4.3"); - tableProperties.put("connector.property-version", "1"); - tableProperties.put("connector.table-name", "testHBastTable"); - tableProperties.put("connector.zookeeper.quorum", "localhost:2181"); - tableProperties.put("connector.zookeeper.znode.parent", "/flink"); - tableProperties.put("connector.write.buffer-flush.max-size", "10mb"); - tableProperties.put("connector.write.buffer-flush.max-rows", "1000"); - tableProperties.put("connector.write.buffer-flush.interval", "10s"); - tableProperties.put("connector.properties.hbase.security.authentication", "kerberos"); - - DescriptorProperties descriptorProperties = new DescriptorProperties(true); - descriptorProperties.putTableSchema(SCHEMA, tableSchema); - descriptorProperties.putProperties(tableProperties); - return descriptorProperties; - } - - @Test - public void testTableSourceFactory() { - TableSchema schema = - TableSchema.builder() - .field(FAMILY1, DataTypes.ROW(DataTypes.FIELD(COL1, DataTypes.INT()))) - .field( - FAMILY2, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.INT()), - DataTypes.FIELD(COL2, DataTypes.BIGINT()))) - .field(ROWKEY, DataTypes.BIGINT()) - .field( - FAMILY3, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.DOUBLE()), - DataTypes.FIELD(COL2, DataTypes.BOOLEAN()), - DataTypes.FIELD(COL3, DataTypes.STRING()))) - .field( - FAMILY4, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.DECIMAL(10, 3)), - DataTypes.FIELD(COL2, DataTypes.TIMESTAMP(3)), - DataTypes.FIELD(COL3, DataTypes.DATE()), - DataTypes.FIELD(COL4, DataTypes.TIME()))) - .build(); - DescriptorProperties descriptorProperties = createDescriptor(schema); - TableSource source = - TableFactoryService.find(HBase1TableFactory.class, descriptorProperties.asMap()) - .createTableSource(descriptorProperties.asMap()); - Assert.assertTrue(source instanceof HBaseTableSource); - TableFunction tableFunction = - ((HBaseTableSource) source).getLookupFunction(new String[] {ROWKEY}); - Assert.assertTrue(tableFunction instanceof HBaseLookupFunction); - Assert.assertEquals( - "testHBastTable", ((HBaseLookupFunction) tableFunction).getHTableName()); - - HBaseTableSchema hbaseSchema = ((HBaseTableSource) source).getHBaseTableSchema(); - Assert.assertEquals(2, hbaseSchema.getRowKeyIndex()); - Assert.assertEquals(Optional.of(Types.LONG), hbaseSchema.getRowKeyTypeInfo()); - - Assert.assertArrayEquals( - new String[] {"f1", "f2", "f3", "f4"}, hbaseSchema.getFamilyNames()); - Assert.assertArrayEquals(new String[] {"c1"}, hbaseSchema.getQualifierNames("f1")); - Assert.assertArrayEquals(new String[] {"c1", "c2"}, hbaseSchema.getQualifierNames("f2")); - Assert.assertArrayEquals( - new String[] {"c1", "c2", "c3"}, hbaseSchema.getQualifierNames("f3")); - Assert.assertArrayEquals( - new String[] {"c1", "c2", "c3", "c4"}, hbaseSchema.getQualifierNames("f4")); - - Assert.assertArrayEquals( - new TypeInformation[] {Types.INT}, hbaseSchema.getQualifierTypes("f1")); - Assert.assertArrayEquals( - new TypeInformation[] {Types.INT, Types.LONG}, hbaseSchema.getQualifierTypes("f2")); - Assert.assertArrayEquals( - new TypeInformation[] {Types.DOUBLE, Types.BOOLEAN, Types.STRING}, - hbaseSchema.getQualifierTypes("f3")); - Assert.assertArrayEquals( - new TypeInformation[] { - Types.BIG_DEC, Types.SQL_TIMESTAMP, Types.SQL_DATE, Types.SQL_TIME - }, - hbaseSchema.getQualifierTypes("f4")); - } - - @Test - public void testTableSinkFactory() { - TableSchema schema = - TableSchema.builder() - .field(ROWKEY, DataTypes.STRING()) - .field( - FAMILY1, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.DOUBLE()), - DataTypes.FIELD(COL2, DataTypes.INT()))) - .field( - FAMILY2, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.INT()), - DataTypes.FIELD(COL3, DataTypes.BIGINT()))) - .field( - FAMILY3, - DataTypes.ROW( - DataTypes.FIELD(COL2, DataTypes.BOOLEAN()), - DataTypes.FIELD(COL3, DataTypes.STRING()))) - .field( - FAMILY4, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.DECIMAL(10, 3)), - DataTypes.FIELD(COL2, DataTypes.TIMESTAMP(3)), - DataTypes.FIELD(COL3, DataTypes.DATE()), - DataTypes.FIELD(COL4, DataTypes.TIME()))) - .build(); - - DescriptorProperties descriptorProperties = createDescriptor(schema); - - TableSink sink = - TableFactoryService.find(HBase1TableFactory.class, descriptorProperties.asMap()) - .createTableSink(descriptorProperties.asMap()); - - Assert.assertTrue(sink instanceof HBaseUpsertTableSink); - - HBaseTableSchema hbaseSchema = ((HBaseUpsertTableSink) sink).getHBaseTableSchema(); - Assert.assertEquals(0, hbaseSchema.getRowKeyIndex()); - Assert.assertEquals(Optional.of(Types.STRING), hbaseSchema.getRowKeyTypeInfo()); - - Assert.assertArrayEquals( - new String[] {"f1", "f2", "f3", "f4"}, hbaseSchema.getFamilyNames()); - Assert.assertArrayEquals(new String[] {"c1", "c2"}, hbaseSchema.getQualifierNames("f1")); - Assert.assertArrayEquals(new String[] {"c1", "c3"}, hbaseSchema.getQualifierNames("f2")); - Assert.assertArrayEquals(new String[] {"c2", "c3"}, hbaseSchema.getQualifierNames("f3")); - Assert.assertArrayEquals( - new String[] {"c1", "c2", "c3", "c4"}, hbaseSchema.getQualifierNames("f4")); - - Assert.assertArrayEquals( - new TypeInformation[] {Types.DOUBLE, Types.INT}, - hbaseSchema.getQualifierTypes("f1")); - Assert.assertArrayEquals( - new TypeInformation[] {Types.INT, Types.LONG}, hbaseSchema.getQualifierTypes("f2")); - Assert.assertArrayEquals( - new TypeInformation[] {Types.BOOLEAN, Types.STRING}, - hbaseSchema.getQualifierTypes("f3")); - Assert.assertArrayEquals( - new TypeInformation[] { - Types.BIG_DEC, Types.SQL_TIMESTAMP, Types.SQL_DATE, Types.SQL_TIME - }, - hbaseSchema.getQualifierTypes("f4")); - - HBaseWriteOptions expectedWriteOptions = - HBaseWriteOptions.builder() - .setBufferFlushMaxRows(1000) - .setBufferFlushIntervalMillis(10 * 1000) - .setBufferFlushMaxSizeInBytes(10 * 1024 * 1024) - .build(); - HBaseWriteOptions actualWriteOptions = ((HBaseUpsertTableSink) sink).getWriteOptions(); - Assert.assertEquals(expectedWriteOptions, actualWriteOptions); - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/example/HBaseWriteExample.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/example/HBaseWriteExample.java deleted file mode 100644 index 397d7d0e2fdae..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/example/HBaseWriteExample.java +++ /dev/null @@ -1,215 +0,0 @@ -/* - * 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.hbase1.example; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.hbase.example.HBaseFlinkTestConstants; -import org.apache.flink.util.Collector; - -import org.apache.hadoop.hbase.client.Mutation; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.mapreduce.TableOutputFormat; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Job; - -/** - * Simple stub for HBase DataSet write - * - *

To run the test first create the test table with hbase shell. - * - *

Use the following commands: - * - *

    - *
  • create 'test-table', 'someCf' - *
- */ -@SuppressWarnings("serial") -public class HBaseWriteExample { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - if (!parseParameters(args)) { - return; - } - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataSet text = getTextDataSet(env); - - DataSet> counts = - // split up the lines in pairs (2-tuples) containing: (word, 1) - text.flatMap(new Tokenizer()) - // group by the tuple field "0" and sum up tuple field "1" - .groupBy(0) - .sum(1); - - // emit result - Job job = Job.getInstance(); - job.getConfiguration().set(TableOutputFormat.OUTPUT_TABLE, outputTableName); - // TODO is "mapred.output.dir" really useful? - job.getConfiguration().set("mapred.output.dir", HBaseFlinkTestConstants.TMP_DIR); - counts.map( - new RichMapFunction, Tuple2>() { - private transient Tuple2 reuse; - - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - reuse = new Tuple2(); - } - - @Override - public Tuple2 map(Tuple2 t) - throws Exception { - reuse.f0 = new Text(t.f0); - Put put = new Put(t.f0.getBytes(ConfigConstants.DEFAULT_CHARSET)); - put.add( - HBaseFlinkTestConstants.CF_SOME, - HBaseFlinkTestConstants.Q_SOME, - Bytes.toBytes(t.f1)); - reuse.f1 = put; - return reuse; - } - }) - .output(new HadoopOutputFormat(new TableOutputFormat(), job)); - - // execute program - env.execute("WordCount (HBase sink) Example"); - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into multiple pairs in the - * form of "(word, 1)" (Tuple2<String, Integer>). - */ - public static final class Tokenizer - implements FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); - } - } - } - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - private static boolean fileOutput = false; - private static String textPath; - private static String outputTableName = HBaseFlinkTestConstants.TEST_TABLE_NAME; - - private static boolean parseParameters(String[] args) { - - if (args.length > 0) { - // parse input arguments - fileOutput = true; - if (args.length == 2) { - textPath = args[0]; - outputTableName = args[1]; - } else { - System.err.println("Usage: HBaseWriteExample "); - return false; - } - } else { - System.out.println("Executing HBaseWriteExample example with built-in default data."); - System.out.println(" Provide parameters to read input data from a file."); - System.out.println(" Usage: HBaseWriteExample "); - } - return true; - } - - private static DataSet getTextDataSet(ExecutionEnvironment env) { - if (fileOutput) { - // read the text file from given input path - return env.readTextFile(textPath); - } else { - // get default test text data - return getDefaultTextLineDataSet(env); - } - } - - private static DataSet getDefaultTextLineDataSet(ExecutionEnvironment env) { - return env.fromElements(WORDS); - } - - private static final String[] WORDS = - new String[] { - "To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", - "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles,", - "And by opposing end them?--To die,--to sleep,--", - "No more; and by a sleep to say we end", - "The heartache, and the thousand natural shocks", - "That flesh is heir to,--'tis a consummation", - "Devoutly to be wish'd. To die,--to sleep;--", - "To sleep! perchance to dream:--ay, there's the rub;", - "For in that sleep of death what dreams may come,", - "When we have shuffled off this mortal coil,", - "Must give us pause: there's the respect", - "That makes calamity of so long life;", - "For who would bear the whips and scorns of time,", - "The oppressor's wrong, the proud man's contumely,", - "The pangs of despis'd love, the law's delay,", - "The insolence of office, and the spurns", - "That patient merit of the unworthy takes,", - "When he himself might his quietus make", - "With a bare bodkin? who would these fardels bear,", - "To grunt and sweat under a weary life,", - "But that the dread of something after death,--", - "The undiscover'd country, from whose bourn", - "No traveller returns,--puzzles the will,", - "And makes us rather bear those ills we have", - "Than fly to others that we know not of?", - "Thus conscience does make cowards of us all;", - "And thus the native hue of resolution", - "Is sicklied o'er with the pale cast of thought;", - "And enterprises of great pith and moment,", - "With this regard, their currents turn awry,", - "And lose the name of action.--Soft you now!", - "The fair Ophelia!--Nymph, in thy orisons", - "Be all my sins remember'd." - }; -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/example/HBaseWriteStreamExample.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/example/HBaseWriteStreamExample.java deleted file mode 100644 index 4feed7b3aec0a..0000000000000 --- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/example/HBaseWriteStreamExample.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.hbase1.example; - -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.SourceFunction; - -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.util.Bytes; - -import java.io.IOException; - -/** - * This is an example how to write streams into HBase. In this example the stream will be written - * into a local Hbase but it is possible to adapt this example for an HBase running in a cloud. You - * need a running local HBase with a table "flinkExample" and a column "entry". If your HBase - * configuration does not fit the hbase-site.xml in the resource folder then you gave to delete - * temporary this hbase-site.xml to execute the example properly. - */ -public class HBaseWriteStreamExample { - - public static void main(String[] args) throws Exception { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // data stream with random numbers - DataStream dataStream = - env.addSource( - new SourceFunction() { - private static final long serialVersionUID = 1L; - - private volatile boolean isRunning = true; - - @Override - public void run(SourceContext out) throws Exception { - while (isRunning) { - out.collect(String.valueOf(Math.floor(Math.random() * 100))); - } - } - - @Override - public void cancel() { - isRunning = false; - } - }); - dataStream.writeUsingOutputFormat(new HBaseOutputFormat()); - - env.execute(); - } - - /** This class implements an OutputFormat for HBase. */ - private static class HBaseOutputFormat implements OutputFormat { - - private org.apache.hadoop.conf.Configuration conf = null; - private HTable table = null; - private String taskNumber = null; - private int rowNumber = 0; - - private static final long serialVersionUID = 1L; - - @Override - public void configure(Configuration parameters) { - conf = HBaseConfiguration.create(); - } - - @Override - public void open(int taskNumber, int numTasks) throws IOException { - table = new HTable(conf, "flinkExample"); - this.taskNumber = String.valueOf(taskNumber); - } - - @Override - public void writeRecord(String record) throws IOException { - Put put = new Put(Bytes.toBytes(taskNumber + rowNumber)); - put.add(Bytes.toBytes("entry"), Bytes.toBytes("entry"), Bytes.toBytes(rowNumber)); - rowNumber++; - table.put(put); - } - - @Override - public void close() throws IOException { - table.flushCommits(); - table.close(); - } - } -} diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java index 23e8e039e93ad..4d267d3043d83 100644 --- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java +++ b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java @@ -18,7 +18,6 @@ package org.apache.flink.connector.hbase1.util; -import org.apache.flink.connector.hbase.util.PlannerType; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.hadoop.hbase.TableName; @@ -80,9 +79,6 @@ public abstract class HBaseTestBase extends HBaseTestingClusterAutoStarter { protected EnvironmentSettings streamSettings; protected EnvironmentSettings batchSettings; - /** Gets the planner type to execute. */ - protected abstract PlannerType planner(); - @BeforeClass public static void activateHBaseCluster() throws IOException { prepareTables(); @@ -93,15 +89,8 @@ public void before() { EnvironmentSettings.Builder streamBuilder = EnvironmentSettings.newInstance().inStreamingMode(); EnvironmentSettings.Builder batchBuilder = EnvironmentSettings.newInstance().inBatchMode(); - if (PlannerType.BLINK_PLANNER.equals(planner())) { - this.streamSettings = streamBuilder.useBlinkPlanner().build(); - this.batchSettings = batchBuilder.useBlinkPlanner().build(); - } else if (PlannerType.OLD_PLANNER.equals(planner())) { - this.streamSettings = streamBuilder.useOldPlanner().build(); - this.batchSettings = batchBuilder.useOldPlanner().build(); - } else { - throw new IllegalArgumentException("Unsupported planner name " + planner()); - } + this.streamSettings = streamBuilder.useBlinkPlanner().build(); + this.batchSettings = batchBuilder.useBlinkPlanner().build(); } private static void prepareTables() throws IOException { diff --git a/flink-connectors/flink-connector-hbase-2.2/README.md b/flink-connectors/flink-connector-hbase-2.2/README.md deleted file mode 100644 index 0ad6d1270373e..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/README.md +++ /dev/null @@ -1,89 +0,0 @@ -# Flink HBase Connector - -This connector provides classes that allow access for Flink to [HBase](https://hbase.apache.org/). - - *Version Compatibility*: This module is compatible with Apache HBase *2.2.3* (last stable version). - -Note that the streaming connectors are not part of the binary distribution of Flink. You need to link them into your job jar for cluster execution. -See how to link with them for cluster execution [here](https://ci.apache.org/projects/flink/flink-docs-release-1.11/dev/project-configuration.html#adding-connector-and-library-dependencies). - -## Installing HBase - -Follow the instructions from the [HBase Quick Start Guide](http://hbase.apache.org/book.html#quickstart). - -## HBase Configuration - -Connecting to HBase always requires a `Configuration` instance. If there is an HBase gateway on the same host as the Flink gateway where the application is started, this can be obtained by invoking `HBaseConfigurationUtil.createHBaseConf()` as in the examples below. If that's not the case a configuration should be provided where the proper core-site, hdfs-site, and hbase-site are added as resources. - -## DataStream API - -### Reading tables into a DataStreams - -To convert an HBase Table into a DataStream one must create an `HBaseTableSource` instance, then either convert it to a `DataStream` of `Row` objects with a built in function, or use the Table API and have a more flexible way to have a stream: - -```java -StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, tableSettings); - -HBaseTableSource hBaseSource = new HBaseTableSource(HBaseConfigurationUtil.createHBaseConf(), "t1"); -hBaseSource.setRowKey("rowkey", byte[].class); -hBaseSource.addColumn("f1", "str", byte[].class); - -// Direct conversion to DataStream -DataStream rowStream = hBaseSource.getDataStream(env); - -// Table API -((TableEnvironmentInternal) tableEnv).registerTableSourceInternal("t1", hBaseSource); -Table table = tableEnv.sqlQuery("SELECT t.rowkey, t.f1.str FROM t1 t"); -DataStream> resultStream = tableEnv.toAppendStream(table, TypeInformation.of(new TypeHint>(){})); -``` - -### Writing into HBase tables from DataStreams -There are two ways to write data to an HBase table from a `DataStream`: -- Instantiate an `HBaseSinkFunction`, and provide one's own `HBaseMutationConverter` implementation that can create mutations from any data received. - -```java -DataStream> dataStream = ... - -HBaseMutationConverter> mutationConverter = new HBaseMutationConverter>() { - private static final long serialVersionUID = 1L; - - @Override - public void open() { - } - - @Override - public Mutation convertToMutation(Tuple2 record) { - Put put = new Put(record.f0); - put.addColumn(Bytes.toBytes("f1"), Bytes.toBytes("str"), record.f1); - return put; - } -}; - -HBaseSinkFunction> hBaseSink = new HBaseSinkFunction>( - "t2", HBaseConfigurationUtil.createHBaseConf(), mutationConverter, 10000, 2, 1000); -dataStream.addSink(hBaseSink); -``` - -- Use the built in `HBaseDynamicTableSink` or `HBaseUpsertTableSink` classes which convert `RowData` or `Tuple2` objects into a mutation each based on an `HBaseTableSchema` provided to them. - -```java -DataStream<> dataStream = ... - -HBaseTableSchema schema = new HBaseTableSchema(); -schema.setRowKey("rowkey", byte[].class); -schema.addColumn("f1", "str", byte[].class); - -HBaseUpsertTableSink sink = new HBaseUpsertTableSink("t3", schema, HBaseConfigurationUtil.createHBaseConf(), - HBaseWriteOptions.builder().setBufferFlushIntervalMillis(1000).build()); -sink.consumeDataStream(dataStream); -``` - -## Building the connector - -The connector can be easily built by using maven: - -``` -cd flink-connectors/flink-connector-hbase -mvn clean install -``` \ No newline at end of file diff --git a/flink-connectors/flink-connector-hbase-2.2/pom.xml b/flink-connectors/flink-connector-hbase-2.2/pom.xml index 7c5d3591d0e17..1e51ddeea0424 100644 --- a/flink-connectors/flink-connector-hbase-2.2/pom.xml +++ b/flink-connectors/flink-connector-hbase-2.2/pom.xml @@ -55,17 +55,7 @@ under the License. - - org.apache.flink - flink-connector-hbase-base_${scala.binary.version} - ${project.version} - - - org.apache.hbase - hbase-server - - - + org.apache.flink @@ -74,34 +64,29 @@ under the License. provided - - org.apache.flink - flink-java - ${project.version} - provided - - - - - - org.apache.flink - flink-scala_${scala.binary.version} + flink-streaming-java_${scala.binary.version} ${project.version} provided - + org.apache.flink - flink-streaming-scala_${scala.binary.version} + flink-connector-hbase-base_${scala.binary.version} ${project.version} - provided + + + org.apache.hbase + hbase-server + + + org.apache.flink @@ -111,12 +96,7 @@ under the License. true - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${project.version} - provided - + org.apache.hadoop @@ -212,7 +192,7 @@ under the License. provided - + org.apache.flink @@ -256,12 +236,14 @@ under the License. + org.apache.flink flink-test-utils_${scala.binary.version} ${project.version} test + org.apache.flink flink-table-common @@ -269,25 +251,14 @@ under the License. test-jar test - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - test - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - test-jar - test - + org.apache.flink flink-table-planner-blink_${scala.binary.version} ${project.version} test + org.apache.flink flink-table-planner-blink_${scala.binary.version} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBase2TableFactory.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBase2TableFactory.java deleted file mode 100644 index 188530c17660d..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBase2TableFactory.java +++ /dev/null @@ -1,226 +0,0 @@ -/* - * 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.annotation.Internal; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.connector.hbase.options.HBaseWriteOptions; -import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.connector.hbase2.sink.HBaseUpsertTableSink; -import org.apache.flink.connector.hbase2.source.HBaseTableSource; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.factories.StreamTableSinkFactory; -import org.apache.flink.table.factories.StreamTableSourceFactory; -import org.apache.flink.table.sinks.StreamTableSink; -import org.apache.flink.table.sources.StreamTableSource; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.types.Row; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; - -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_PROPERTIES; -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_TABLE_NAME; -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_TYPE_VALUE_HBASE; -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_VERSION_VALUE_223; -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL; -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS; -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE; -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_ZK_NODE_PARENT; -import static org.apache.flink.connector.hbase2.HBaseValidator.CONNECTOR_ZK_QUORUM; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_DATA_TYPE; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE; - -/** Factory for creating configured instances of {@link HBaseTableSource} or sink. */ -@Internal -public class HBase2TableFactory - implements StreamTableSourceFactory, StreamTableSinkFactory> { - - @Override - public StreamTableSource createStreamTableSource(Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - Configuration hbaseClientConf = getHBaseConf(descriptorProperties); - - String hTableName = descriptorProperties.getString(CONNECTOR_TABLE_NAME); - TableSchema tableSchema = - TableSchemaUtils.getPhysicalSchema(descriptorProperties.getTableSchema(SCHEMA)); - HBaseTableSchema hbaseSchema = validateTableSchema(tableSchema); - return new HBaseTableSource(hbaseClientConf, hTableName, hbaseSchema, null); - } - - @Override - public StreamTableSink> createStreamTableSink( - Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - - TableSchema tableSchema = - TableSchemaUtils.getPhysicalSchema(descriptorProperties.getTableSchema(SCHEMA)); - HBaseTableSchema hbaseSchema = validateTableSchema(tableSchema); - - Configuration hbaseClientConf = getHBaseConf(descriptorProperties); - - HBaseWriteOptions.Builder writeBuilder = HBaseWriteOptions.builder(); - descriptorProperties - .getOptionalInt(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS) - .ifPresent(writeBuilder::setBufferFlushMaxRows); - descriptorProperties - .getOptionalMemorySize(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE) - .ifPresent(v -> writeBuilder.setBufferFlushMaxSizeInBytes(v.getBytes())); - descriptorProperties - .getOptionalDuration(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL) - .ifPresent(v -> writeBuilder.setBufferFlushIntervalMillis(v.toMillis())); - - return new HBaseUpsertTableSink( - descriptorProperties.getString(CONNECTOR_TABLE_NAME), - hbaseSchema, - hbaseClientConf, - writeBuilder.build()); - } - - private HBaseTableSchema validateTableSchema(TableSchema schema) { - HBaseTableSchema hbaseSchema = new HBaseTableSchema(); - String[] fieldNames = schema.getFieldNames(); - TypeInformation[] fieldTypes = schema.getFieldTypes(); - for (int i = 0; i < fieldNames.length; i++) { - String name = fieldNames[i]; - TypeInformation type = fieldTypes[i]; - if (type instanceof RowTypeInfo) { - RowTypeInfo familyType = (RowTypeInfo) type; - String[] qualifierNames = familyType.getFieldNames(); - TypeInformation[] qualifierTypes = familyType.getFieldTypes(); - for (int j = 0; j < familyType.getArity(); j++) { - // HBase connector doesn't support LocalDateTime - // use Timestamp as conversion class for now. - Class clazz = qualifierTypes[j].getTypeClass(); - if (LocalDateTime.class.equals(clazz)) { - clazz = Timestamp.class; - } else if (LocalDate.class.equals(clazz)) { - clazz = Date.class; - } else if (LocalTime.class.equals(clazz)) { - clazz = Time.class; - } - hbaseSchema.addColumn(name, qualifierNames[j], clazz); - } - } else { - hbaseSchema.setRowKey(name, type.getTypeClass()); - } - } - return hbaseSchema; - } - - private DescriptorProperties getValidatedProperties(Map properties) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(true); - descriptorProperties.putProperties(properties); - new HBaseValidator().validate(descriptorProperties); - return descriptorProperties; - } - - @Override - public Map requiredContext() { - Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_HBASE); // hbase - context.put(CONNECTOR_VERSION, hbaseVersion()); // version - context.put(CONNECTOR_PROPERTY_VERSION, "1"); // backwards compatibility - return context; - } - - @Override - public List supportedProperties() { - List properties = new ArrayList<>(); - - properties.add(CONNECTOR_VERSION); - properties.add(CONNECTOR_TABLE_NAME); - properties.add(CONNECTOR_ZK_QUORUM); - properties.add(CONNECTOR_ZK_NODE_PARENT); - properties.add(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE); - properties.add(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS); - properties.add(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL); - - // schema - properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); - properties.add(SCHEMA + ".#." + SCHEMA_TYPE); - properties.add(SCHEMA + ".#." + SCHEMA_NAME); - // computed column - properties.add(SCHEMA + ".#." + EXPR); - - // watermark - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_ROWTIME); - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_EXPR); - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_DATA_TYPE); - - // table constraint - properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_NAME); - properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_COLUMNS); - - // comment - properties.add(COMMENT); - - // HBase properties - properties.add(CONNECTOR_PROPERTIES + ".*"); - - return properties; - } - - private String hbaseVersion() { - return CONNECTOR_VERSION_VALUE_223; - } - - private static Configuration getHBaseConf(DescriptorProperties descriptorProperties) { - Configuration hbaseClientConf = HBaseConfigurationUtil.createHBaseConf(); - descriptorProperties - .getOptionalString(CONNECTOR_ZK_QUORUM) - .ifPresent(zkQ -> hbaseClientConf.set(HConstants.ZOOKEEPER_QUORUM, zkQ)); - - descriptorProperties - .getOptionalString(CONNECTOR_ZK_NODE_PARENT) - .ifPresent(v -> hbaseClientConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, v)); - - // add HBase properties - descriptorProperties - .getPropertiesWithPrefix(CONNECTOR_PROPERTIES) - .forEach(hbaseClientConf::set); - return hbaseClientConf; - } -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBaseValidator.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBaseValidator.java deleted file mode 100644 index 3093c1c4ddfe1..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBaseValidator.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.annotation.Internal; -import org.apache.flink.table.descriptors.AbstractHBaseValidator; - -/** - * The validator for HBase. More features to be supported, e.g., batch read/write, async api(support - * from hbase version 2.0.0), Caching for LookupFunction. - */ -@Internal -public class HBaseValidator extends AbstractHBaseValidator { - public static final String CONNECTOR_VERSION_VALUE_223 = "2.2.3"; - - @Override - protected boolean validateZkQuorum() { - return true; - } - - @Override - protected String getConnectorVersion() { - return CONNECTOR_VERSION_VALUE_223; - } - - @Override - protected boolean zkQuorumIsOptional() { - return true; - } -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/sink/HBaseUpsertTableSink.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/sink/HBaseUpsertTableSink.java deleted file mode 100644 index 60c2c15669ffb..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/sink/HBaseUpsertTableSink.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.sink; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.connector.hbase.options.HBaseWriteOptions; -import org.apache.flink.connector.hbase.sink.HBaseSinkFunction; -import org.apache.flink.connector.hbase.sink.LegacyMutationConverter; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.sinks.UpsertStreamTableSink; -import org.apache.flink.table.utils.TableConnectorUtils; -import org.apache.flink.types.Row; - -import org.apache.hadoop.conf.Configuration; - -import java.util.Arrays; - -import static org.apache.flink.util.Preconditions.checkArgument; - -/** An upsert {@link UpsertStreamTableSink} for HBase. */ -@Internal -public class HBaseUpsertTableSink implements UpsertStreamTableSink { - - private final HBaseTableSchema hbaseTableSchema; - private final TableSchema tableSchema; - private final Configuration hconf; - private final HBaseWriteOptions writeOptions; - private final String tableName; - - public HBaseUpsertTableSink( - String tableName, - HBaseTableSchema hbaseTableSchema, - Configuration hconf, - HBaseWriteOptions writeOptions) { - checkArgument( - hbaseTableSchema.getRowKeyName().isPresent(), - "HBaseUpsertTableSink requires rowkey is set."); - this.hbaseTableSchema = hbaseTableSchema; - this.tableSchema = hbaseTableSchema.convertsToTableSchema(); - this.hconf = hconf; - this.writeOptions = writeOptions; - this.tableName = tableName; - } - - @Override - public void setKeyFields(String[] keys) { - // hbase always upsert on rowkey, ignore query keys. - // Actually, we should verify the query key is the same with rowkey. - // However, the query key extraction doesn't work well in some scenarios - // (e.g. concat key fields will lose key information). So we skip key validation currently. - } - - @Override - public void setIsAppendOnly(Boolean isAppendOnly) { - // hbase always upsert on rowkey, even works in append only mode. - } - - @Override - public TypeInformation getRecordType() { - return tableSchema.toRowType(); - } - - @Override - public TableSchema getTableSchema() { - return tableSchema; - } - - @Override - public DataStreamSink consumeDataStream(DataStream> dataStream) { - HBaseSinkFunction sinkFunction = - new HBaseSinkFunction( - tableName, - hconf, - new LegacyMutationConverter(hbaseTableSchema), - writeOptions.getBufferFlushMaxSizeInBytes(), - writeOptions.getBufferFlushMaxRows(), - writeOptions.getBufferFlushIntervalMillis()); - return dataStream - .addSink(sinkFunction) - .setParallelism(dataStream.getParallelism()) - .name( - TableConnectorUtils.generateRuntimeName( - this.getClass(), tableSchema.getFieldNames())); - } - - @Override - public TableSink> configure( - String[] fieldNames, TypeInformation[] fieldTypes) { - if (!Arrays.equals(getFieldNames(), fieldNames) - || !Arrays.equals(getFieldTypes(), fieldTypes)) { - throw new ValidationException( - "Reconfiguration with different fields is not allowed. " - + "Expected: " - + Arrays.toString(getFieldNames()) - + " / " - + Arrays.toString(getFieldTypes()) - + ". " - + "But was: " - + Arrays.toString(fieldNames) - + " / " - + Arrays.toString(fieldTypes)); - } - - return new HBaseUpsertTableSink(tableName, hbaseTableSchema, hconf, writeOptions); - } - - @VisibleForTesting - public HBaseTableSchema getHBaseTableSchema() { - return hbaseTableSchema; - } - - @VisibleForTesting - public HBaseWriteOptions getWriteOptions() { - return writeOptions; - } -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseInputFormat.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseInputFormat.java deleted file mode 100644 index e8ea17811c3c2..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseInputFormat.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.Experimental; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.java.tuple.Tuple; - -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; - -/** {@link InputFormat} subclass that wraps the access for HTables. */ -@Experimental -public abstract class HBaseInputFormat extends AbstractTableInputFormat { - - private static final long serialVersionUID = 1L; - - /** - * Constructs a {@link InputFormat} with hbase configuration to read data from hbase. - * - * @param hConf The configuration that connect to hbase. At least hbase.zookeeper.quorum and - * zookeeper.znode.parent need to be set. - */ - public HBaseInputFormat(org.apache.hadoop.conf.Configuration hConf) { - super(hConf); - } - - /** - * Returns an instance of Scan that retrieves the required subset of records from the HBase - * table. - * - * @return The appropriate instance of Scan for this usecase. - */ - @Override - protected abstract Scan getScanner(); - - /** - * What table is to be read. Per instance of a TableInputFormat derivative only a single - * tablename is possible. - * - * @return The name of the table - */ - @Override - protected abstract String getTableName(); - - /** - * The output from HBase is always an instance of {@link Result}. This method is to copy the - * data in the Result instance into the required {@link Tuple} - * - * @param r The Result instance from HBase that needs to be converted - * @return The appropriate instance of {@link Tuple} that contains the needed information. - */ - protected abstract T mapResultToTuple(Result r); - - @Override - protected void initTable() { - if (table == null) { - createTable(); - } - if (table != null && scan == null) { - scan = getScanner(); - } - } - - /** Create an {@link HTable} instance and set it into this format. */ - private void createTable() { - try { - if (connection == null) { - this.connection = ConnectionFactory.createConnection(getHadoopConfiguration()); - } - TableName name = TableName.valueOf(getTableName()); - table = connection.getTable(name); - regionLocator = connection.getRegionLocator(name); - } catch (TableNotFoundException tnfe) { - LOG.error("The table " + table.getName().getNameAsString() + " not found ", tnfe); - throw new RuntimeException( - "HBase table '" + table.getName().getNameAsString() + "' not found.", tnfe); - } catch (Exception e) { - throw new RuntimeException("Error connecting to the HBase table", e); - } - } - - @Override - protected T mapResultToOutType(Result r) { - return mapResultToTuple(r); - } -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowInputFormat.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowInputFormat.java deleted file mode 100644 index 31ae127c4c9df..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowInputFormat.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.api.common.io.InputFormat; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.connector.hbase.util.HBaseReadWriteHelper; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.types.Row; - -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; - -/** - * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link Row} - */ -@Internal -public class HBaseRowInputFormat extends AbstractTableInputFormat - implements ResultTypeQueryable { - - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(HBaseRowInputFormat.class); - - private final String tableName; - private final HBaseTableSchema schema; - - private transient HBaseReadWriteHelper readHelper; - - public HBaseRowInputFormat( - org.apache.hadoop.conf.Configuration conf, String tableName, HBaseTableSchema schema) { - super(conf); - this.tableName = tableName; - this.schema = schema; - } - - @Override - public void initTable() throws IOException { - this.readHelper = new HBaseReadWriteHelper(schema); - if (table == null) { - connectToTable(); - } - if (table != null && scan == null) { - scan = getScanner(); - } - } - - @Override - protected Scan getScanner() { - return readHelper.createScan(); - } - - @Override - public String getTableName() { - return tableName; - } - - @Override - protected Row mapResultToOutType(Result res) { - return readHelper.parseToRow(res); - } - - private void connectToTable() throws IOException { - try { - if (connection == null) { - this.connection = ConnectionFactory.createConnection(getHadoopConfiguration()); - } - TableName name = TableName.valueOf(getTableName()); - table = connection.getTable(name); - regionLocator = connection.getRegionLocator(name); - } catch (TableNotFoundException tnfe) { - LOG.error("The table " + tableName + " not found ", tnfe); - throw new RuntimeException("HBase table '" + tableName + "' not found.", tnfe); - } - } - - @Override - public TypeInformation getProducedType() { - // split the fieldNames - String[] famNames = schema.getFamilyNames(); - TypeInformation[] typeInfos = new TypeInformation[famNames.length]; - int i = 0; - for (String family : famNames) { - typeInfos[i] = - new RowTypeInfo( - schema.getQualifierTypes(family), schema.getQualifierNames(family)); - i++; - } - return new RowTypeInfo(typeInfos, famNames); - } -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseTableSource.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseTableSource.java deleted file mode 100644 index 6fc5adae322cc..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseTableSource.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.api.common.io.InputFormat; -import org.apache.flink.connector.hbase.source.AbstractHBaseTableSource; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.types.Row; - -import org.apache.hadoop.conf.Configuration; - -/** - * Creates a TableSource to scan an HBase table. - * - *

The table name and required HBase configuration is passed during {@link HBaseTableSource} - * construction. Use {@link #addColumn(String, String, Class)} to specify the family, qualifier, and - * type of columns to scan. - * - *

The TableSource returns {@link Row} with nested Rows for each column family. - * - *

The HBaseTableSource is used as shown in the example below. - * - *

{@code
- * HBaseTableSource hSrc = new HBaseTableSource(conf, "hTable");
- * hSrc.setRowKey("rowkey", String.class);
- * hSrc.addColumn("fam1", "col1", byte[].class);
- * hSrc.addColumn("fam1", "col2", Integer.class);
- * hSrc.addColumn("fam2", "col1", String.class);
- *
- * tableEnv.registerTableSourceInternal("hTable", hSrc);
- * Table res = tableEnv.sqlQuery(
- *   "SELECT t.fam2.col1, SUM(t.fam1.col2) FROM hTable AS t " +
- *   "WHERE t.rowkey LIKE 'flink%' GROUP BY t.fam2.col1");
- * }
- */ -@Internal -public class HBaseTableSource extends AbstractHBaseTableSource { - - /** - * The HBase configuration and the name of the table to read. - * - * @param conf hbase configuration - * @param tableName the tableName - */ - public HBaseTableSource(Configuration conf, String tableName) { - this(conf, tableName, new HBaseTableSchema(), null); - } - - public HBaseTableSource( - Configuration conf, - String tableName, - HBaseTableSchema hbaseSchema, - int[] projectFields) { - super(conf, tableName, hbaseSchema, projectFields); - } - - @Override - public HBaseTableSource projectFields(int[] fields) { - return new HBaseTableSource(conf, tableName, hbaseSchema, fields); - } - - @Override - public InputFormat getInputFormat(HBaseTableSchema projectedSchema) { - return new HBaseRowInputFormat(conf, tableName, projectedSchema); - } -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-hbase-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index 56a70858ae43b..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ /dev/null @@ -1,16 +0,0 @@ -# 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.hbase2.HBase2TableFactory diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java index 33ab7c2e98408..d9633161cafad 100644 --- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java +++ b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java @@ -18,32 +18,18 @@ 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; @@ -52,12 +38,8 @@ 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; @@ -65,70 +47,38 @@ 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) +/** IT cases for HBase connector (including source and sink). */ 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," - + " family2 ROW," - + " family3 ROW," - + " rowkey INT," - + " PRIMARY KEY (rowkey) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'hbase-2.2'," - + " 'table-name' = '" - + TEST_TABLE_1 - + "'," - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "'" - + ")"); - } + TableEnvironment tEnv = TableEnvironment.create(batchSettings); + + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW," + + " family2 ROW," + + " family3 ROW," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + + TEST_TABLE_1 + + "'," + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "'" + + ")"); Table table = tEnv.sqlQuery( @@ -157,36 +107,24 @@ public void testTableSourceFullScan() { @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," - + " family2 ROW," - + " family3 ROW," - + " rowkey INT," - + " PRIMARY KEY (rowkey) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'hbase-2.2'," - + " 'table-name' = '" - + TEST_TABLE_1 - + "'," - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "'" - + ")"); - } + TableEnvironment tEnv = TableEnvironment.create(batchSettings); + + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family1 ROW," + + " family2 ROW," + + " family3 ROW," + + " rowkey INT," + + " PRIMARY KEY (rowkey) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + + TEST_TABLE_1 + + "'," + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "'" + + ")"); Table table = tEnv.sqlQuery( @@ -213,36 +151,23 @@ public void testTableSourceProjection() { @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," - + " family3 ROW," - + " family1 ROW" - + ") WITH (" - + " 'connector' = 'hbase-2.2'," - + " 'table-name' = '" - + TEST_TABLE_1 - + "'," - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "'" - + ")"); - } + TableEnvironment tEnv = TableEnvironment.create(batchSettings); + + tEnv.executeSql( + "CREATE TABLE hTable (" + + " rowkey INT PRIMARY KEY," + + " family2 ROW," + + " family3 ROW," + + " family1 ROW" + + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + + TEST_TABLE_1 + + "'," + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "'" + + ")"); Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h"); @@ -261,74 +186,23 @@ public void testTableSourceFieldOrder() { } @Test - public void testTableSourceWithTableAPI() throws Exception { - StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); - tEnv.connect( - new HBase() - .version("2.2.3") - .tableName(TEST_TABLE_1) - .zookeeperQuorum(getZookeeperQuorum())) - .withSchema( - new Schema() - .field("rowkey", DataTypes.INT()) - .field( - "family2", - DataTypes.ROW( - DataTypes.FIELD("col1", DataTypes.STRING()), - DataTypes.FIELD("col2", DataTypes.BIGINT()))) - .field( - "family3", - DataTypes.ROW( - DataTypes.FIELD("col1", DataTypes.DOUBLE()), - DataTypes.FIELD("col2", DataTypes.BOOLEAN()), - DataTypes.FIELD("col3", DataTypes.STRING()))) - .field( - "family1", - DataTypes.ROW(DataTypes.FIELD("col1", DataTypes.INT())))) - .createTemporaryTable("hTable"); - Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h"); - List results = CollectionUtil.iteratorToList(table.execute().collect()); - String expected = - "+I[1, +I[Hello-1, 100], +I[1.01, false, Welt-1], +I[10]]\n" - + "+I[2, +I[Hello-2, 200], +I[2.02, true, Welt-2], +I[20]]\n" - + "+I[3, +I[Hello-3, 300], +I[3.03, false, Welt-3], +I[30]]\n" - + "+I[4, +I[null, 400], +I[4.04, true, Welt-4], +I[40]]\n" - + "+I[5, +I[Hello-5, 500], +I[5.05, false, Welt-5], +I[50]]\n" - + "+I[6, +I[Hello-6, 600], +I[6.06, true, Welt-6], +I[60]]\n" - + "+I[7, +I[Hello-7, 700], +I[7.07, false, Welt-7], +I[70]]\n" - + "+I[8, +I[null, 800], +I[8.08, true, Welt-8], +I[80]]\n"; + public void testTableSourceReadAsByteArray() { + TableEnvironment tEnv = TableEnvironment.create(batchSettings); - TestBaseUtils.compareResultAsText(results, expected); - } - - @Test - public void testTableSourceReadAsByteArray() throws Exception { - TableEnvironment tEnv = createBatchTableEnv(); - - if (isLegacyConnector) { - // fetch row2 from the table till the end - HBaseTableSource hbaseTable = new HBaseTableSource(getConf(), TEST_TABLE_1); - hbaseTable.addColumn(FAMILY2, F2COL1, byte[].class); - hbaseTable.addColumn(FAMILY2, F2COL2, byte[].class); - hbaseTable.setRowKey(ROW_KEY, Integer.class); - ((TableEnvironmentInternal) tEnv).registerTableSourceInternal("hTable", hbaseTable); - } else { - tEnv.executeSql( - "CREATE TABLE hTable (" - + " family2 ROW," - + " rowkey INT" - + // no primary key syntax - ") WITH (" - + " 'connector' = 'hbase-2.2'," - + " 'table-name' = '" - + TEST_TABLE_1 - + "'," - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "'" - + ")"); - } + tEnv.executeSql( + "CREATE TABLE hTable (" + + " family2 ROW," + + " rowkey INT" + + // no primary key syntax + ") WITH (" + + " 'connector' = 'hbase-2.2'," + + " 'table-name' = '" + + TEST_TABLE_1 + + "'," + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "'" + + ")"); tEnv.registerFunction("toUTF8", new ToUTF8()); tEnv.registerFunction("toLong", new ToLong()); @@ -353,22 +227,6 @@ public void testTableSourceReadAsByteArray() throws Exception { TestBaseUtils.compareResultAsText(results, expected); } - @Test - public void testTableInputFormat() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> result = - env.createInput(new InputFormatForTestTable(getConf())) - .reduce( - (ReduceFunction>) - (v1, v2) -> Tuple1.of(v1.f0 + v2.f0)); - - List> resultSet = result.collect(); - - assertEquals(1, resultSet.size()); - assertEquals(360, (int) resultSet.get(0).f0); - } - @Test public void testTableSink() throws Exception { StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -396,7 +254,7 @@ public void testTableSink() throws Exception { tEnv.executeSql(query).await(); // start a batch scan job to verify contents in HBase table - TableEnvironment batchEnv = createBatchTableEnv(); + TableEnvironment batchEnv = TableEnvironment.create(batchSettings); batchEnv.executeSql(table2DDL); Table table = @@ -428,14 +286,6 @@ public void testTableSink() throws Exception { @Test public void testTableSourceSinkWithDDL() throws Exception { - if (OLD_PLANNER.equals(planner) || isLegacyConnector) { - // only test for blink planner and new connector, because types - // TIMESTAMP/DATE/TIME/DECIMAL works well in - // new connector(using blink-planner), but exits some precision problem in old planner - // or legacy connector. - return; - } - StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); @@ -461,7 +311,7 @@ public void testTableSourceSinkWithDDL() throws Exception { tEnv.executeSql(insertStatement).await(); // start a batch scan job to verify contents in HBase table - TableEnvironment batchEnv = createBatchTableEnv(); + TableEnvironment batchEnv = TableEnvironment.create(batchSettings); batchEnv.executeSql(table3DDL); String query = "SELECT " @@ -520,12 +370,8 @@ public void testHBaseAsyncLookupTableSource() { public void testTableInputFormatOpenClose() throws IOException { HBaseTableSchema tableSchema = new HBaseTableSchema(); tableSchema.addColumn(FAMILY1, F1COL1, byte[].class); - AbstractTableInputFormat inputFormat; - if (isLegacyConnector) { - inputFormat = new HBaseRowInputFormat(getConf(), TEST_TABLE_1, tableSchema); - } else { - inputFormat = new HBaseRowDataInputFormat(getConf(), TEST_TABLE_1, tableSchema, "null"); - } + AbstractTableInputFormat inputFormat = + new HBaseRowDataInputFormat(getConf(), TEST_TABLE_1, tableSchema, "null"); inputFormat.open(inputFormat.createInputSplits(1)[0]); assertNotNull(inputFormat.getConnection()); assertNotNull(inputFormat.getConnection().getTable(TableName.valueOf(TEST_TABLE_1))); @@ -535,12 +381,6 @@ public void testTableInputFormatOpenClose() throws IOException { } private void verifyHBaseLookupJoin(boolean async) { - if (OLD_PLANNER.equals(planner) || isLegacyConnector) { - // lookup table source is only supported in blink planner, skip for old planner - // types TIMESTAMP/DATE/TIME/DECIMAL works well in new connector, skip legacy connector - return; - } - StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); @@ -631,16 +471,6 @@ private void verifyHBaseLookupJoin(boolean async) { // ------------------------------- Utilities ------------------------------------------------- - /** Creates a Batch {@link TableEnvironment} depends on the {@link #planner} context. */ - private TableEnvironment createBatchTableEnv() { - if (OLD_PLANNER.equals(planner)) { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - return BatchTableEnvironment.create(env, new TableConfig()); - } else { - return TableEnvironment.create(batchSettings); - } - } - /** A {@link ScalarFunction} that maps byte arrays to UTF-8 strings. */ public static class ToUTF8 extends ScalarFunction { private static final long serialVersionUID = 1L; @@ -659,31 +489,6 @@ public long eval(byte[] bytes) { } } - /** A {@link HBaseInputFormat} for testing. */ - public static class InputFormatForTestTable extends HBaseInputFormat> { - private static final long serialVersionUID = 1L; - - public InputFormatForTestTable(org.apache.hadoop.conf.Configuration hConf) { - super(hConf); - } - - @Override - protected Scan getScanner() { - return new Scan(); - } - - @Override - protected String getTableName() { - return TEST_TABLE_1; - } - - @Override - protected Tuple1 mapResultToTuple(Result r) { - return new Tuple1<>( - Bytes.toInt(r.getValue(Bytes.toBytes(FAMILY1), Bytes.toBytes(F1COL1)))); - } - } - private String createHBaseTableDDL(String tableName, boolean testTimeAndDecimalTypes) { StringBuilder family4Statement = new StringBuilder(); if (testTimeAndDecimalTypes) { @@ -693,45 +498,24 @@ private String createHBaseTableDDL(String tableName, boolean testTimeAndDecimalT family4Statement.append(", col4 DECIMAL(12, 4)"); family4Statement.append("> \n"); } - if (isLegacyConnector) { - return "CREATE TABLE " - + tableName - + "(\n" - + " rowkey INT,\n" - + " family1 ROW,\n" - + " family2 ROW,\n" - + " family3 ROW" - + family4Statement.toString() - + ") WITH (\n" - + " 'connector.type' = 'hbase',\n" - + " 'connector.version' = '2.2.3',\n" - + " 'connector.table-name' = '" - + tableName - + "',\n" - + " 'connector.zookeeper.quorum' = '" - + getZookeeperQuorum() - + "',\n" - + " 'connector.zookeeper.znode.parent' = '/hbase' " - + ")"; - } else { - return "CREATE TABLE " - + tableName - + "(\n" - + " rowkey INT," - + " family1 ROW,\n" - + " family2 ROW,\n" - + " family3 ROW" - + family4Statement.toString() - + ") WITH (\n" - + " 'connector' = 'hbase-2.2',\n" - + " 'table-name' = '" - + tableName - + "',\n" - + " 'zookeeper.quorum' = '" - + getZookeeperQuorum() - + "',\n" - + " 'zookeeper.znode.parent' = '/hbase' " - + ")"; - } + + return "CREATE TABLE " + + tableName + + "(\n" + + " rowkey INT," + + " family1 ROW,\n" + + " family2 ROW,\n" + + " family3 ROW" + + family4Statement.toString() + + ") WITH (\n" + + " 'connector' = 'hbase-2.2',\n" + + " 'table-name' = '" + + tableName + + "',\n" + + " 'zookeeper.quorum' = '" + + getZookeeperQuorum() + + "',\n" + + " 'zookeeper.znode.parent' = '/hbase' " + + ")"; } } diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseDescriptorTest.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseDescriptorTest.java deleted file mode 100644 index 4eb43c00e51e1..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseDescriptorTest.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * 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.table.api.DataTypes; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.internal.Registration; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.descriptors.ConnectTableDescriptor; -import org.apache.flink.table.descriptors.Descriptor; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.DescriptorTestBase; -import org.apache.flink.table.descriptors.DescriptorValidator; -import org.apache.flink.table.descriptors.FormatDescriptor; -import org.apache.flink.table.descriptors.HBase; -import org.apache.flink.table.descriptors.Rowtime; -import org.apache.flink.table.descriptors.Schema; -import org.apache.flink.table.descriptors.StreamTableDescriptor; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; - -/** Test case for {@link HBase} descriptor. */ -public class HBaseDescriptorTest extends DescriptorTestBase { - - @Override - protected List descriptors() { - HBase hbaseDesc0 = - new HBase() - .version("2.2.3") - .tableName("testNs:table0") - .zookeeperQuorum("localhost:2181,localhost:2182,localhost:2183") - .zookeeperNodeParent("/hbase/root-dir"); - - HBase hbaseDesc1 = - new HBase() - .version("2.2.3") - .tableName("testNs:table1") - .zookeeperQuorum("localhost:2181") - .zookeeperNodeParent("/hbase/root") - .writeBufferFlushInterval("2s") - .writeBufferFlushMaxRows(100) - .writeBufferFlushMaxSize("1mb"); - - return Arrays.asList(hbaseDesc0, hbaseDesc1); - } - - @Override - protected List> properties() { - Map prop0 = new HashMap<>(); - prop0.put("connector.version", "2.2.3"); - prop0.put("connector.type", "hbase"); - prop0.put("connector.table-name", "testNs:table0"); - prop0.put("connector.zookeeper.quorum", "localhost:2181,localhost:2182,localhost:2183"); - prop0.put("connector.zookeeper.znode.parent", "/hbase/root-dir"); - prop0.put("connector.property-version", "1"); - - Map prop1 = new HashMap<>(); - prop1.put("connector.version", "2.2.3"); - prop1.put("connector.type", "hbase"); - prop1.put("connector.table-name", "testNs:table1"); - prop1.put("connector.zookeeper.quorum", "localhost:2181"); - prop1.put("connector.zookeeper.znode.parent", "/hbase/root"); - prop1.put("connector.property-version", "1"); - prop1.put("connector.write.buffer-flush.interval", "2s"); - prop1.put("connector.write.buffer-flush.max-rows", "100"); - prop1.put("connector.write.buffer-flush.max-size", "1 mb"); - - return Arrays.asList(prop0, prop1); - } - - @Override - protected DescriptorValidator validator() { - return new HBaseValidator(); - } - - @Test - public void testRequiredFields() { - HBase hbaseDesc0 = new HBase(); - HBase hbaseDesc1 = - new HBase() - .version("2.2.3") - .zookeeperQuorum("localhost:2181") - .zookeeperNodeParent("/hbase/root"); // no table name - - HBase[] testCases = new HBase[] {hbaseDesc0, hbaseDesc1}; - for (int i = 0; i < testCases.length; i++) { - HBase hbaseDesc = testCases[i]; - DescriptorProperties properties = new DescriptorProperties(); - properties.putProperties(hbaseDesc.toProperties()); - boolean caughtExpectedException = false; - try { - validator().validate(properties); - } catch (ValidationException e) { - caughtExpectedException = true; - } - Assert.assertTrue( - "The case#" + i + " didn't get the expected error", caughtExpectedException); - } - } - - @Test - public void testFormatNeed() { - String expected = - "The connector org.apache.flink.table.descriptors.HBase does not require a format description but org.apache.flink.connector.hbase2.HBaseDescriptorTest$1 found."; - AtomicReference reference = new AtomicReference<>(); - HBase hBase = new HBase(); - Registration registration = (path, table) -> reference.set((CatalogTableImpl) table); - ConnectTableDescriptor descriptor = - new StreamTableDescriptor(registration, hBase) - .withFormat( - new FormatDescriptor("myFormat", 1) { - @Override - protected Map toFormatProperties() { - return new HashMap<>(); - } - }) - .withSchema( - new Schema() - .field("f0", DataTypes.INT()) - .rowtime(new Rowtime().timestampsFromField("f0"))); - String actual = null; - try { - descriptor.toProperties(); - } catch (Exception e) { - actual = e.getMessage(); - } - Assert.assertEquals(expected, actual); - } -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseTableFactoryTest.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseTableFactoryTest.java deleted file mode 100644 index dd61f32a6bfbe..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseTableFactoryTest.java +++ /dev/null @@ -1,214 +0,0 @@ -/* - * Copyright The Apache Software Foundation - * - * 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.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.connector.hbase.options.HBaseWriteOptions; -import org.apache.flink.connector.hbase.source.HBaseLookupFunction; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.connector.hbase2.sink.HBaseUpsertTableSink; -import org.apache.flink.connector.hbase2.source.HBaseTableSource; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.factories.TableFactoryService; -import org.apache.flink.table.functions.TableFunction; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.sources.TableSource; -import org.apache.flink.types.Row; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; - -import static org.apache.flink.table.descriptors.Schema.SCHEMA; - -/** UT for HBaseTableFactory. */ -public class HBaseTableFactoryTest { - private static final String FAMILY1 = "f1"; - private static final String FAMILY2 = "f2"; - private static final String FAMILY3 = "f3"; - private static final String FAMILY4 = "f4"; - private static final String COL1 = "c1"; - private static final String COL2 = "c2"; - private static final String COL3 = "c3"; - private static final String COL4 = "c4"; - private static final String ROWKEY = "rowkey"; - - private DescriptorProperties createDescriptor(TableSchema tableSchema) { - Map tableProperties = new HashMap<>(); - tableProperties.put("connector.type", "hbase"); - tableProperties.put("connector.version", "2.2.3"); - tableProperties.put("connector.property-version", "1"); - tableProperties.put("connector.table-name", "testHBastTable"); - tableProperties.put("connector.zookeeper.quorum", "localhost:2181"); - tableProperties.put("connector.zookeeper.znode.parent", "/flink"); - tableProperties.put("connector.write.buffer-flush.max-size", "10mb"); - tableProperties.put("connector.write.buffer-flush.max-rows", "1000"); - tableProperties.put("connector.write.buffer-flush.interval", "10s"); - tableProperties.put("connector.properties.hbase.security.authentication", "kerberos"); - - DescriptorProperties descriptorProperties = new DescriptorProperties(true); - descriptorProperties.putTableSchema(SCHEMA, tableSchema); - descriptorProperties.putProperties(tableProperties); - return descriptorProperties; - } - - @Test - public void testTableSourceFactory() { - TableSchema schema = - TableSchema.builder() - .field(FAMILY1, DataTypes.ROW(DataTypes.FIELD(COL1, DataTypes.INT()))) - .field( - FAMILY2, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.INT()), - DataTypes.FIELD(COL2, DataTypes.BIGINT()))) - .field(ROWKEY, DataTypes.BIGINT()) - .field( - FAMILY3, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.DOUBLE()), - DataTypes.FIELD(COL2, DataTypes.BOOLEAN()), - DataTypes.FIELD(COL3, DataTypes.STRING()))) - .field( - FAMILY4, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.DECIMAL(10, 3)), - DataTypes.FIELD(COL2, DataTypes.TIMESTAMP(3)), - DataTypes.FIELD(COL3, DataTypes.DATE()), - DataTypes.FIELD(COL4, DataTypes.TIME()))) - .build(); - DescriptorProperties descriptorProperties = createDescriptor(schema); - TableSource source = - TableFactoryService.find(HBase2TableFactory.class, descriptorProperties.asMap()) - .createTableSource(descriptorProperties.asMap()); - Assert.assertTrue(source instanceof HBaseTableSource); - TableFunction tableFunction = - ((HBaseTableSource) source).getLookupFunction(new String[] {ROWKEY}); - Assert.assertTrue(tableFunction instanceof HBaseLookupFunction); - Assert.assertEquals( - "testHBastTable", ((HBaseLookupFunction) tableFunction).getHTableName()); - - HBaseTableSchema hbaseSchema = ((HBaseTableSource) source).getHBaseTableSchema(); - Assert.assertEquals(2, hbaseSchema.getRowKeyIndex()); - Assert.assertEquals(Optional.of(Types.LONG), hbaseSchema.getRowKeyTypeInfo()); - - Assert.assertArrayEquals( - new String[] {"f1", "f2", "f3", "f4"}, hbaseSchema.getFamilyNames()); - Assert.assertArrayEquals(new String[] {"c1"}, hbaseSchema.getQualifierNames("f1")); - Assert.assertArrayEquals(new String[] {"c1", "c2"}, hbaseSchema.getQualifierNames("f2")); - Assert.assertArrayEquals( - new String[] {"c1", "c2", "c3"}, hbaseSchema.getQualifierNames("f3")); - Assert.assertArrayEquals( - new String[] {"c1", "c2", "c3", "c4"}, hbaseSchema.getQualifierNames("f4")); - - Assert.assertArrayEquals( - new TypeInformation[] {Types.INT}, hbaseSchema.getQualifierTypes("f1")); - Assert.assertArrayEquals( - new TypeInformation[] {Types.INT, Types.LONG}, hbaseSchema.getQualifierTypes("f2")); - Assert.assertArrayEquals( - new TypeInformation[] {Types.DOUBLE, Types.BOOLEAN, Types.STRING}, - hbaseSchema.getQualifierTypes("f3")); - Assert.assertArrayEquals( - new TypeInformation[] { - Types.BIG_DEC, Types.SQL_TIMESTAMP, Types.SQL_DATE, Types.SQL_TIME - }, - hbaseSchema.getQualifierTypes("f4")); - } - - @Test - public void testTableSinkFactory() { - TableSchema schema = - TableSchema.builder() - .field(ROWKEY, DataTypes.STRING()) - .field( - FAMILY1, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.DOUBLE()), - DataTypes.FIELD(COL2, DataTypes.INT()))) - .field( - FAMILY2, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.INT()), - DataTypes.FIELD(COL3, DataTypes.BIGINT()))) - .field( - FAMILY3, - DataTypes.ROW( - DataTypes.FIELD(COL2, DataTypes.BOOLEAN()), - DataTypes.FIELD(COL3, DataTypes.STRING()))) - .field( - FAMILY4, - DataTypes.ROW( - DataTypes.FIELD(COL1, DataTypes.DECIMAL(10, 3)), - DataTypes.FIELD(COL2, DataTypes.TIMESTAMP(3)), - DataTypes.FIELD(COL3, DataTypes.DATE()), - DataTypes.FIELD(COL4, DataTypes.TIME()))) - .build(); - - DescriptorProperties descriptorProperties = createDescriptor(schema); - - TableSink sink = - TableFactoryService.find(HBase2TableFactory.class, descriptorProperties.asMap()) - .createTableSink(descriptorProperties.asMap()); - - Assert.assertTrue(sink instanceof HBaseUpsertTableSink); - - HBaseTableSchema hbaseSchema = ((HBaseUpsertTableSink) sink).getHBaseTableSchema(); - Assert.assertEquals(0, hbaseSchema.getRowKeyIndex()); - Assert.assertEquals(Optional.of(Types.STRING), hbaseSchema.getRowKeyTypeInfo()); - - Assert.assertArrayEquals( - new String[] {"f1", "f2", "f3", "f4"}, hbaseSchema.getFamilyNames()); - Assert.assertArrayEquals(new String[] {"c1", "c2"}, hbaseSchema.getQualifierNames("f1")); - Assert.assertArrayEquals(new String[] {"c1", "c3"}, hbaseSchema.getQualifierNames("f2")); - Assert.assertArrayEquals(new String[] {"c2", "c3"}, hbaseSchema.getQualifierNames("f3")); - Assert.assertArrayEquals( - new String[] {"c1", "c2", "c3", "c4"}, hbaseSchema.getQualifierNames("f4")); - - Assert.assertArrayEquals( - new TypeInformation[] {Types.DOUBLE, Types.INT}, - hbaseSchema.getQualifierTypes("f1")); - Assert.assertArrayEquals( - new TypeInformation[] {Types.INT, Types.LONG}, hbaseSchema.getQualifierTypes("f2")); - Assert.assertArrayEquals( - new TypeInformation[] {Types.BOOLEAN, Types.STRING}, - hbaseSchema.getQualifierTypes("f3")); - Assert.assertArrayEquals( - new TypeInformation[] { - Types.BIG_DEC, Types.SQL_TIMESTAMP, Types.SQL_DATE, Types.SQL_TIME - }, - hbaseSchema.getQualifierTypes("f4")); - - HBaseWriteOptions expectedWriteOptions = - HBaseWriteOptions.builder() - .setBufferFlushMaxRows(1000) - .setBufferFlushIntervalMillis(10 * 1000) - .setBufferFlushMaxSizeInBytes(10 * 1024 * 1024) - .build(); - HBaseWriteOptions actualWriteOptions = ((HBaseUpsertTableSink) sink).getWriteOptions(); - Assert.assertEquals(expectedWriteOptions, actualWriteOptions); - } -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/example/HBaseWriteExample.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/example/HBaseWriteExample.java deleted file mode 100644 index 53dadf85dff1e..0000000000000 --- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/example/HBaseWriteExample.java +++ /dev/null @@ -1,215 +0,0 @@ -/* - * 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.example; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.hbase.example.HBaseFlinkTestConstants; -import org.apache.flink.util.Collector; - -import org.apache.hadoop.hbase.client.Mutation; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.mapreduce.TableOutputFormat; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Job; - -/** - * Simple stub for HBase DataSet write - * - *

To run the test first create the test table with hbase shell. - * - *

Use the following commands: - * - *

    - *
  • create 'test-table', 'someCf' - *
- */ -@SuppressWarnings("serial") -public class HBaseWriteExample { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - if (!parseParameters(args)) { - return; - } - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataSet text = getTextDataSet(env); - - DataSet> counts = - // split up the lines in pairs (2-tuples) containing: (word, 1) - text.flatMap(new Tokenizer()) - // group by the tuple field "0" and sum up tuple field "1" - .groupBy(0) - .sum(1); - - // emit result - Job job = Job.getInstance(); - job.getConfiguration().set(TableOutputFormat.OUTPUT_TABLE, outputTableName); - // TODO is "mapred.output.dir" really useful? - job.getConfiguration().set("mapred.output.dir", HBaseFlinkTestConstants.TMP_DIR); - counts.map( - new RichMapFunction, Tuple2>() { - private transient Tuple2 reuse; - - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - reuse = new Tuple2(); - } - - @Override - public Tuple2 map(Tuple2 t) - throws Exception { - reuse.f0 = new Text(t.f0); - Put put = new Put(t.f0.getBytes(ConfigConstants.DEFAULT_CHARSET)); - put.addColumn( - HBaseFlinkTestConstants.CF_SOME, - HBaseFlinkTestConstants.Q_SOME, - Bytes.toBytes(t.f1)); - reuse.f1 = put; - return reuse; - } - }) - .output(new HadoopOutputFormat(new TableOutputFormat(), job)); - - // execute program - env.execute("WordCount (HBase sink) Example"); - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into multiple pairs in the - * form of "(word, 1)" (Tuple2<String, Integer>). - */ - public static final class Tokenizer - implements FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); - } - } - } - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - private static boolean fileOutput = false; - private static String textPath; - private static String outputTableName = HBaseFlinkTestConstants.TEST_TABLE_NAME; - - private static boolean parseParameters(String[] args) { - - if (args.length > 0) { - // parse input arguments - fileOutput = true; - if (args.length == 2) { - textPath = args[0]; - outputTableName = args[1]; - } else { - System.err.println("Usage: HBaseWriteExample "); - return false; - } - } else { - System.out.println("Executing HBaseWriteExample example with built-in default data."); - System.out.println(" Provide parameters to read input data from a file."); - System.out.println(" Usage: HBaseWriteExample "); - } - return true; - } - - private static DataSet getTextDataSet(ExecutionEnvironment env) { - if (fileOutput) { - // read the text file from given input path - return env.readTextFile(textPath); - } else { - // get default test text data - return getDefaultTextLineDataSet(env); - } - } - - private static DataSet getDefaultTextLineDataSet(ExecutionEnvironment env) { - return env.fromElements(WORDS); - } - - private static final String[] WORDS = - new String[] { - "To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", - "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles,", - "And by opposing end them?--To die,--to sleep,--", - "No more; and by a sleep to say we end", - "The heartache, and the thousand natural shocks", - "That flesh is heir to,--'tis a consummation", - "Devoutly to be wish'd. To die,--to sleep;--", - "To sleep! perchance to dream:--ay, there's the rub;", - "For in that sleep of death what dreams may come,", - "When we have shuffled off this mortal coil,", - "Must give us pause: there's the respect", - "That makes calamity of so long life;", - "For who would bear the whips and scorns of time,", - "The oppressor's wrong, the proud man's contumely,", - "The pangs of despis'd love, the law's delay,", - "The insolence of office, and the spurns", - "That patient merit of the unworthy takes,", - "When he himself might his quietus make", - "With a bare bodkin? who would these fardels bear,", - "To grunt and sweat under a weary life,", - "But that the dread of something after death,--", - "The undiscover'd country, from whose bourn", - "No traveller returns,--puzzles the will,", - "And makes us rather bear those ills we have", - "Than fly to others that we know not of?", - "Thus conscience does make cowards of us all;", - "And thus the native hue of resolution", - "Is sicklied o'er with the pale cast of thought;", - "And enterprises of great pith and moment,", - "With this regard, their currents turn awry,", - "And lose the name of action.--Soft you now!", - "The fair Ophelia!--Nymph, in thy orisons", - "Be all my sins remember'd." - }; -} diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunctionTest.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunctionTest.java index 6c6c7993d61e5..38a9273da197b 100644 --- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunctionTest.java +++ b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunctionTest.java @@ -20,7 +20,6 @@ import org.apache.flink.connector.hbase.options.HBaseLookupOptions; import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.connector.hbase.util.PlannerType; import org.apache.flink.connector.hbase2.util.HBaseTestBase; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; @@ -52,12 +51,6 @@ public static Object[] parameters() { return new Object[][] {new Object[] {true}, new Object[] {false}}; } - @Override - protected PlannerType planner() { - // lookup table source is only supported in blink planner - return PlannerType.BLINK_PLANNER; - } - @Test public void testEval() throws Exception { HBaseRowDataAsyncLookupFunction lookupFunction = buildRowDataAsyncLookupFunction(); diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java index 50d1cf869a6e7..1fbf11b8a2578 100644 --- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java +++ b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java @@ -18,7 +18,6 @@ package org.apache.flink.connector.hbase2.util; -import org.apache.flink.connector.hbase.util.PlannerType; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.hadoop.hbase.TableName; @@ -80,9 +79,6 @@ public abstract class HBaseTestBase extends HBaseTestingClusterAutoStarter { protected EnvironmentSettings streamSettings; protected EnvironmentSettings batchSettings; - /** Gets the planner type to execute. */ - protected abstract PlannerType planner(); - @BeforeClass public static void activateHBaseCluster() throws IOException { prepareTables(); @@ -93,15 +89,8 @@ public void before() { EnvironmentSettings.Builder streamBuilder = EnvironmentSettings.newInstance().inStreamingMode(); EnvironmentSettings.Builder batchBuilder = EnvironmentSettings.newInstance().inBatchMode(); - if (PlannerType.BLINK_PLANNER.equals(planner())) { - this.streamSettings = streamBuilder.useBlinkPlanner().build(); - this.batchSettings = batchBuilder.useBlinkPlanner().build(); - } else if (PlannerType.OLD_PLANNER.equals(planner())) { - this.streamSettings = streamBuilder.useOldPlanner().build(); - this.batchSettings = batchBuilder.useOldPlanner().build(); - } else { - throw new IllegalArgumentException("Unsupported planner name " + planner()); - } + this.streamSettings = streamBuilder.useBlinkPlanner().build(); + this.batchSettings = batchBuilder.useBlinkPlanner().build(); } private static void prepareTables() throws IOException { diff --git a/flink-connectors/flink-connector-hbase-base/pom.xml b/flink-connectors/flink-connector-hbase-base/pom.xml index e046521da24c7..b0e8ead71e8e7 100644 --- a/flink-connectors/flink-connector-hbase-base/pom.xml +++ b/flink-connectors/flink-connector-hbase-base/pom.xml @@ -39,41 +39,24 @@ under the License. - - org.apache.flink - flink-core - ${project.version} - provided - - - - - - org.apache.flink - flink-java - ${project.version} - provided - - - + org.apache.flink - flink-scala_${scala.binary.version} + flink-core ${project.version} provided - - org.apache.flink - flink-streaming-scala_${scala.binary.version} + flink-streaming-java_${scala.binary.version} ${project.version} provided + org.apache.flink @@ -83,12 +66,7 @@ under the License. true - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${project.version} - provided - + org.apache.hadoop diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/LegacyMutationConverter.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/LegacyMutationConverter.java deleted file mode 100644 index 8d0dacfb3c5b3..0000000000000 --- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/LegacyMutationConverter.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.sink; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.connector.hbase.util.HBaseReadWriteHelper; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.types.Row; - -import org.apache.hadoop.hbase.client.Mutation; - -/** - * Legacy implementation for {@link - * org.apache.flink.connector.hbase.source.AbstractHBaseTableSource}. - */ -public class LegacyMutationConverter implements HBaseMutationConverter> { - private static final long serialVersionUID = 7358222494016900667L; - - private final HBaseTableSchema schema; - - private transient HBaseReadWriteHelper helper; - - public LegacyMutationConverter(HBaseTableSchema schema) { - this.schema = schema; - } - - @Override - public void open() { - this.helper = new HBaseReadWriteHelper(schema); - } - - @Override - public Mutation convertToMutation(Tuple2 record) { - if (record.f0) { - return helper.createPutMutation(record.f1); - } else { - return helper.createDeleteMutation(record.f1); - } - } -} diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/AbstractHBaseTableSource.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/AbstractHBaseTableSource.java deleted file mode 100644 index 5ac9f7d91c64d..0000000000000 --- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/AbstractHBaseTableSource.java +++ /dev/null @@ -1,194 +0,0 @@ -/* - * 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.api.common.io.InputFormat; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.functions.AsyncTableFunction; -import org.apache.flink.table.functions.TableFunction; -import org.apache.flink.table.sources.BatchTableSource; -import org.apache.flink.table.sources.LookupableTableSource; -import org.apache.flink.table.sources.ProjectableTableSource; -import org.apache.flink.table.sources.StreamTableSource; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; - -import org.apache.hadoop.conf.Configuration; - -import java.util.Arrays; - -/** - * Creates a TableSource to scan an HBase table. - * - *

The table name and required HBase configuration is passed during {@link - * AbstractHBaseTableSource} construction. Use {@link #addColumn(String, String, Class)} to specify - * the family, qualifier, and type of columns to scan. - * - *

The TableSource returns {@link Row} with nested Rows for each column family. - * - *

The HBaseTableSource is used as shown in the example below. - * - *

{@code
- * HBaseTableSource hSrc = new HBaseTableSource(conf, "hTable");
- * hSrc.setRowKey("rowkey", String.class);
- * hSrc.addColumn("fam1", "col1", byte[].class);
- * hSrc.addColumn("fam1", "col2", Integer.class);
- * hSrc.addColumn("fam2", "col1", String.class);
- *
- * tableEnv.registerTableSourceInternal("hTable", hSrc);
- * Table res = tableEnv.sqlQuery(
- *   "SELECT t.fam2.col1, SUM(t.fam1.col2) FROM hTable AS t " +
- *   "WHERE t.rowkey LIKE 'flink%' GROUP BY t.fam2.col1");
- * }
- */ -@Internal -public abstract class AbstractHBaseTableSource - implements BatchTableSource, - ProjectableTableSource, - StreamTableSource, - LookupableTableSource { - - protected final Configuration conf; - protected final String tableName; - protected final HBaseTableSchema hbaseSchema; - private final int[] projectFields; - - public AbstractHBaseTableSource( - Configuration conf, - String tableName, - HBaseTableSchema hbaseSchema, - int[] projectFields) { - this.conf = conf; - this.tableName = Preconditions.checkNotNull(tableName, "Table name"); - this.hbaseSchema = hbaseSchema; - this.projectFields = projectFields; - } - - /** - * Adds a column defined by family, qualifier, and type to the table schema. - * - * @param family the family name - * @param qualifier the qualifier name - * @param clazz the data type of the qualifier - */ - public void addColumn(String family, String qualifier, Class clazz) { - this.hbaseSchema.addColumn(family, qualifier, clazz); - } - - /** - * Sets row key information in the table schema. - * - * @param rowKeyName the row key field name - * @param clazz the data type of the row key - */ - public void setRowKey(String rowKeyName, Class clazz) { - this.hbaseSchema.setRowKey(rowKeyName, clazz); - } - - /** - * Specifies the charset to parse Strings to HBase byte[] keys and String values. - * - * @param charset Name of the charset to use. - */ - public void setCharset(String charset) { - this.hbaseSchema.setCharset(charset); - } - - @Override - public TypeInformation getReturnType() { - HBaseTableSchema projectedSchema = hbaseSchema.getProjectedHBaseTableSchema(projectFields); - return projectedSchema.convertsToTableSchema().toRowType(); - } - - @Override - public TableSchema getTableSchema() { - return hbaseSchema.convertsToTableSchema(); - } - - @Override - public DataSet getDataSet(ExecutionEnvironment execEnv) { - HBaseTableSchema projectedSchema = hbaseSchema.getProjectedHBaseTableSchema(projectFields); - return execEnv.createInput(getInputFormat(projectedSchema), getReturnType()) - .name(explainSource()); - } - - @Override - public String explainSource() { - return "HBaseTableSource[schema=" - + Arrays.toString(getTableSchema().getFieldNames()) - + ", projectFields=" - + Arrays.toString(projectFields) - + "]"; - } - - @Override - public TableFunction getLookupFunction(String[] lookupKeys) { - Preconditions.checkArgument( - null != lookupKeys && lookupKeys.length == 1, - "HBase table can only be retrieved by rowKey for now."); - Preconditions.checkState( - hbaseSchema.getRowKeyName().isPresent(), - "HBase schema must have a row key when used in lookup mode."); - Preconditions.checkState( - hbaseSchema.getRowKeyName().get().equals(lookupKeys[0]), - "The lookup key is not row key of HBase."); - - return new HBaseLookupFunction( - this.conf, this.tableName, hbaseSchema.getProjectedHBaseTableSchema(projectFields)); - } - - @Override - public AsyncTableFunction getAsyncLookupFunction(String[] lookupKeys) { - throw new UnsupportedOperationException( - "HBase table doesn't support async lookup currently."); - } - - @Override - public boolean isAsyncEnabled() { - return false; - } - - @Override - public boolean isBounded() { - // HBase source is always bounded. - return true; - } - - @Override - public DataStream getDataStream(StreamExecutionEnvironment execEnv) { - HBaseTableSchema projectedSchema = hbaseSchema.getProjectedHBaseTableSchema(projectFields); - return execEnv.createInput(getInputFormat(projectedSchema), getReturnType()) - .name(explainSource()); - } - - protected abstract InputFormat getInputFormat(HBaseTableSchema projectedSchema); - - @VisibleForTesting - public HBaseTableSchema getHBaseTableSchema() { - return this.hbaseSchema; - } -} diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/HBaseLookupFunction.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/HBaseLookupFunction.java deleted file mode 100644 index 692484cca93db..0000000000000 --- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/HBaseLookupFunction.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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.api.common.typeinfo.TypeInformation; -import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; -import org.apache.flink.connector.hbase.util.HBaseReadWriteHelper; -import org.apache.flink.connector.hbase.util.HBaseTableSchema; -import org.apache.flink.table.functions.FunctionContext; -import org.apache.flink.table.functions.TableFunction; -import org.apache.flink.types.Row; -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 HBaseLookupFunction 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 Row}. - */ -@Internal -public class HBaseLookupFunction extends TableFunction { - private static final Logger LOG = LoggerFactory.getLogger(HBaseLookupFunction.class); - private static final long serialVersionUID = 1L; - - private final String hTableName; - private final byte[] serializedConfig; - private final HBaseTableSchema hbaseTableSchema; - - private transient HBaseReadWriteHelper readHelper; - private transient Connection hConnection; - private transient HTable table; - - public HBaseLookupFunction( - Configuration configuration, String hTableName, HBaseTableSchema hbaseTableSchema) { - this.serializedConfig = HBaseConfigurationUtil.serializeConfiguration(configuration); - this.hTableName = hTableName; - this.hbaseTableSchema = hbaseTableSchema; - } - - /** - * 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)); - if (!result.isEmpty()) { - // parse and collect - collect(readHelper.parseToRow(result, rowKey)); - } - } - - @Override - public TypeInformation getResultType() { - return hbaseTableSchema.convertsToTableSchema().toRowType(); - } - - private org.apache.hadoop.conf.Configuration prepareRuntimeConfiguration() { - // create default configuration from current runtime env (`hbase-site.xml` in classpath) - // first, - // and overwrite configuration using serialized configuration from client-side env - // (`hbase-site.xml` in classpath). - // user params from client-side have the highest priority - org.apache.hadoop.conf.Configuration runtimeConfig = - HBaseConfigurationUtil.deserializeConfiguration( - serializedConfig, HBaseConfigurationUtil.getHBaseConfiguration()); - - // do validation: check key option(s) in final runtime configuration - if (StringUtils.isNullOrWhitespaceOnly(runtimeConfig.get(HConstants.ZOOKEEPER_QUORUM))) { - LOG.error( - "can not connect to HBase without {} configuration", - HConstants.ZOOKEEPER_QUORUM); - throw new IllegalArgumentException( - "check HBase configuration failed, lost: '" - + HConstants.ZOOKEEPER_QUORUM - + "'!"); - } - - return runtimeConfig; - } - - @Override - public void open(FunctionContext context) { - LOG.info("start open ..."); - org.apache.hadoop.conf.Configuration config = prepareRuntimeConfiguration(); - try { - hConnection = ConnectionFactory.createConnection(config); - table = (HTable) hConnection.getTable(TableName.valueOf(hTableName)); - } catch (TableNotFoundException tnfe) { - LOG.error("Table '{}' not found ", hTableName, tnfe); - throw new RuntimeException("HBase table '" + hTableName + "' not found.", tnfe); - } catch (IOException ioe) { - LOG.error("Exception while creating connection to HBase.", ioe); - throw new RuntimeException("Cannot create connection to HBase.", ioe); - } - this.readHelper = new HBaseReadWriteHelper(hbaseTableSchema); - LOG.info("end open."); - } - - @Override - public void close() { - LOG.info("start close ..."); - if (null != table) { - try { - table.close(); - table = null; - } catch (IOException e) { - // ignore exception when close. - LOG.warn("exception when close table", e); - } - } - if (null != hConnection) { - try { - hConnection.close(); - hConnection = null; - } catch (IOException e) { - // ignore exception when close. - LOG.warn("exception when close connection", e); - } - } - LOG.info("end close."); - } - - @VisibleForTesting - public String getHTableName() { - return hTableName; - } -} diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseReadWriteHelper.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseReadWriteHelper.java deleted file mode 100644 index fa413df2266b7..0000000000000 --- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseReadWriteHelper.java +++ /dev/null @@ -1,225 +0,0 @@ -/* - * 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.util; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; - -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; - -import java.nio.charset.Charset; - -/** - * A read and write helper for HBase. The helper can used to create a {@link Scan} and {@link Get} - * for scanning or lookuping a HBase table, and create a {@link Put} and {@link Delete} for writing - * to HBase table, and supports converting the HBase {@link Result} to Flink {@link Row}. - */ -@Internal -public class HBaseReadWriteHelper { - - // family keys - private final byte[][] families; - // qualifier keys - private final byte[][][] qualifiers; - // qualifier types - private final int[][] qualifierTypes; - - // row key index in output row - private final int rowKeyIndex; - // type of row key - private final int rowKeyType; - - private final int fieldLength; - - // charset is not serializable - private final Charset charset; - - // row which is returned - private Row resultRow; - // nested family rows - private Row[] familyRows; - - public HBaseReadWriteHelper(HBaseTableSchema hbaseTableSchema) { - this.families = hbaseTableSchema.getFamilyKeys(); - this.qualifiers = new byte[this.families.length][][]; - this.qualifierTypes = new int[this.families.length][]; - this.familyRows = new Row[this.families.length]; - String[] familyNames = hbaseTableSchema.getFamilyNames(); - for (int f = 0; f < families.length; f++) { - this.qualifiers[f] = hbaseTableSchema.getQualifierKeys(familyNames[f]); - TypeInformation[] typeInfos = hbaseTableSchema.getQualifierTypes(familyNames[f]); - this.qualifierTypes[f] = new int[typeInfos.length]; - for (int i = 0; i < typeInfos.length; i++) { - qualifierTypes[f][i] = HBaseTypeUtils.getTypeIndex(typeInfos[i]); - } - this.familyRows[f] = new Row(typeInfos.length); - } - this.charset = Charset.forName(hbaseTableSchema.getStringCharset()); - // row key - this.rowKeyIndex = hbaseTableSchema.getRowKeyIndex(); - this.rowKeyType = - hbaseTableSchema.getRowKeyTypeInfo().map(HBaseTypeUtils::getTypeIndex).orElse(-1); - - // field length need take row key into account if it exists. - this.fieldLength = rowKeyIndex == -1 ? families.length : families.length + 1; - - // prepare output rows - this.resultRow = new Row(fieldLength); - } - - /** - * Returns an instance of Get that retrieves the matches records from the HBase table. - * - * @return The appropriate instance of Get for this use case. - */ - public Get createGet(Object rowKey) { - byte[] rowkey = HBaseTypeUtils.serializeFromObject(rowKey, rowKeyType, charset); - Get get = new Get(rowkey); - for (int f = 0; f < families.length; f++) { - byte[] family = families[f]; - for (byte[] qualifier : qualifiers[f]) { - get.addColumn(family, qualifier); - } - } - return get; - } - - /** - * Returns an instance of Scan that retrieves the required subset of records from the HBase - * table. - * - * @return The appropriate instance of Scan for this use case. - */ - public Scan createScan() { - Scan scan = new Scan(); - for (int f = 0; f < families.length; f++) { - byte[] family = families[f]; - for (int q = 0; q < qualifiers[f].length; q++) { - byte[] quantifier = qualifiers[f][q]; - scan.addColumn(family, quantifier); - } - } - return scan; - } - - /** Parses HBase {@link Result} into {@link Row}. */ - public Row parseToRow(Result result) { - if (rowKeyIndex == -1) { - return parseToRow(result, null); - } else { - Object rowkey = - HBaseTypeUtils.deserializeToObject(result.getRow(), rowKeyType, charset); - return parseToRow(result, rowkey); - } - } - - /** Parses HBase {@link Result} into {@link Row}. */ - public Row parseToRow(Result result, Object rowKey) { - for (int i = 0; i < fieldLength; i++) { - if (rowKeyIndex == i) { - resultRow.setField(rowKeyIndex, rowKey); - } else { - int f = (rowKeyIndex != -1 && i > rowKeyIndex) ? i - 1 : i; - // get family key - byte[] familyKey = families[f]; - Row familyRow = familyRows[f]; - for (int q = 0; q < this.qualifiers[f].length; q++) { - // get quantifier key - byte[] qualifier = qualifiers[f][q]; - // get quantifier type idx - int typeIdx = qualifierTypes[f][q]; - // read value - byte[] value = result.getValue(familyKey, qualifier); - if (value != null) { - familyRow.setField( - q, HBaseTypeUtils.deserializeToObject(value, typeIdx, charset)); - } else { - familyRow.setField(q, null); - } - } - resultRow.setField(i, familyRow); - } - } - return resultRow; - } - - /** - * Returns an instance of Put that writes record to HBase table. - * - * @return The appropriate instance of Put for this use case. - */ - public Put createPutMutation(Row row) { - assert rowKeyIndex != -1; - byte[] rowkey = - HBaseTypeUtils.serializeFromObject(row.getField(rowKeyIndex), rowKeyType, charset); - // upsert - Put put = new Put(rowkey); - for (int i = 0; i < fieldLength; i++) { - if (i != rowKeyIndex) { - int f = i > rowKeyIndex ? i - 1 : i; - // get family key - byte[] familyKey = families[f]; - Row familyRow = (Row) row.getField(i); - for (int q = 0; q < this.qualifiers[f].length; q++) { - // get quantifier key - byte[] qualifier = qualifiers[f][q]; - // get quantifier type idx - int typeIdx = qualifierTypes[f][q]; - // read value - byte[] value = - HBaseTypeUtils.serializeFromObject( - familyRow.getField(q), typeIdx, charset); - put.addColumn(familyKey, qualifier, value); - } - } - } - return put; - } - - /** - * Returns an instance of Delete that remove record from HBase table. - * - * @return The appropriate instance of Delete for this use case. - */ - public Delete createDeleteMutation(Row row) { - assert rowKeyIndex != -1; - byte[] rowkey = - HBaseTypeUtils.serializeFromObject(row.getField(rowKeyIndex), rowKeyType, charset); - // delete - Delete delete = new Delete(rowkey); - for (int i = 0; i < fieldLength; i++) { - if (i != rowKeyIndex) { - int f = i > rowKeyIndex ? i - 1 : i; - // get family key - byte[] familyKey = families[f]; - for (int q = 0; q < this.qualifiers[f].length; q++) { - // get quantifier key - byte[] qualifier = qualifiers[f][q]; - delete.addColumn(familyKey, qualifier); - } - } - } - return delete; - } -} diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/AbstractHBaseValidator.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/AbstractHBaseValidator.java deleted file mode 100644 index 538c903aff14a..0000000000000 --- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/AbstractHBaseValidator.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.descriptors; - -import org.apache.flink.annotation.Internal; - -import java.util.Arrays; -import java.util.List; - -/** - * The validator for HBase. More features to be supported, e.g., batch read/write, async api(support - * from hbase version 2.0.0), Caching for LookupFunction. - */ -@Internal -public abstract class AbstractHBaseValidator extends ConnectorDescriptorValidator { - - public static final String CONNECTOR_TYPE_VALUE_HBASE = "hbase"; - public static final String CONNECTOR_TABLE_NAME = "connector.table-name"; - public static final String CONNECTOR_ZK_QUORUM = "connector.zookeeper.quorum"; - public static final String CONNECTOR_ZK_NODE_PARENT = "connector.zookeeper.znode.parent"; - public static final String CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE = - "connector.write.buffer-flush.max-size"; - public static final String CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS = - "connector.write.buffer-flush.max-rows"; - public static final String CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL = - "connector.write.buffer-flush.interval"; - public static final String CONNECTOR_PROPERTIES = "connector.properties"; - - @Override - 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, validateZkQuorum(), 1); - properties.validateString(CONNECTOR_ZK_NODE_PARENT, true, 1); - validateSinkProperties(properties); - validateVersion(properties); - } - - protected abstract boolean validateZkQuorum(); - - private void validateSinkProperties(DescriptorProperties properties) { - properties.validateMemorySize( - CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE, - true, - 1024 * 1024); // only allow MB precision - properties.validateInt(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS, true, 1); - properties.validateDuration(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL, true, 1); - } - - private void validateVersion(DescriptorProperties properties) { - final List versions = Arrays.asList(getConnectorVersion()); - properties.validateEnumValues(CONNECTOR_VERSION, zkQuorumIsOptional(), versions); - } - - protected abstract String getConnectorVersion(); - - protected abstract boolean zkQuorumIsOptional(); -} diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/HBase.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/HBase.java deleted file mode 100644 index 440a3cdf81860..0000000000000 --- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/table/descriptors/HBase.java +++ /dev/null @@ -1,173 +0,0 @@ -/* - * 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.descriptors; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.TimeUtils; - -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import static org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_PROPERTIES; -import static org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_TABLE_NAME; -import static org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_TYPE_VALUE_HBASE; -import static org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL; -import static org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS; -import static org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE; -import static org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_NODE_PARENT; -import static org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_QUORUM; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; - -/** Connector descriptor for Apache HBase. */ -@PublicEvolving -public class HBase extends ConnectorDescriptor { - private DescriptorProperties properties = new DescriptorProperties(); - private Map hbaseProperties; - - public HBase() { - super(CONNECTOR_TYPE_VALUE_HBASE, 1, false); - } - - /** - * Set the Apache HBase version to be used. Required. - * - * @param version HBase version. E.g., "1.4.3". - */ - public HBase version(String version) { - properties.putString(CONNECTOR_VERSION, version); - return this; - } - - /** - * Set the HBase table name, Required. - * - * @param tableName Name of HBase table. E.g., "testNamespace:testTable", "testDefaultTable" - */ - public HBase tableName(String tableName) { - properties.putString(CONNECTOR_TABLE_NAME, tableName); - return this; - } - - /** - * Set the zookeeper quorum address to connect the HBase cluster. Required. - * - * @param zookeeperQuorum zookeeper quorum address to connect the HBase cluster. E.g., - * "localhost:2181,localhost:2182,localhost:2183". - */ - public HBase zookeeperQuorum(String zookeeperQuorum) { - properties.putString(CONNECTOR_ZK_QUORUM, zookeeperQuorum); - return this; - } - - /** - * Set the zookeeper node parent path of HBase cluster. Default to use "/hbase", Optional. - * - * @param zookeeperNodeParent zookeeper node path of hbase cluster. E.g, - * "/hbase/example-root-znode". - */ - public HBase zookeeperNodeParent(String zookeeperNodeParent) { - properties.putString(CONNECTOR_ZK_NODE_PARENT, zookeeperNodeParent); - return this; - } - - /** - * Set threshold when to flush buffered request based on the memory byte size of rows currently - * added. Default to 2mb. Optional. - * - * @param maxSize the maximum size (using the syntax of {@link MemorySize}). - */ - public HBase writeBufferFlushMaxSize(String maxSize) { - properties.putMemorySize( - CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE, - MemorySize.parse(maxSize, MemorySize.MemoryUnit.BYTES)); - return this; - } - - /** - * Set threshold when to flush buffered request based on the number of rows currently added. - * Defaults to not set, i.e. won't flush based on the number of buffered rows. Optional. - * - * @param writeBufferFlushMaxRows number of added rows when begin the request flushing. - */ - public HBase writeBufferFlushMaxRows(int writeBufferFlushMaxRows) { - properties.putInt(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS, writeBufferFlushMaxRows); - return this; - } - - /** - * Set an interval when to flushing buffered requesting if the interval passes, in milliseconds. - * Defaults to not set, i.e. won't flush based on flush interval. Optional. - * - * @param interval flush interval. The string should be in format "{length value}{time unit - * label}" E.g, "123ms", "1 s", If no time unit label is specified, it will be considered as - * milliseconds. For more details about the format, please see {@link - * TimeUtils#parseDuration(String)}}. - */ - public HBase writeBufferFlushInterval(String interval) { - properties.putString(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL, interval); - return this; - } - - /** - * Sets the configuration properties for HBase Configuration. Resets previously set properties. - * - * @param properties The configuration properties for HBase Configuration. - */ - public HBase properties(Properties properties) { - Preconditions.checkNotNull(properties); - if (this.hbaseProperties == null) { - this.hbaseProperties = new HashMap<>(); - } - this.hbaseProperties.clear(); - properties.forEach((k, v) -> this.hbaseProperties.put((String) k, (String) v)); - return this; - } - - /** - * Adds a configuration property for HBase Configuration. - * - * @param key property key for the HBase Configuration - * @param value property value for the HBase Configuration - */ - public HBase property(String key, String value) { - Preconditions.checkNotNull(key); - Preconditions.checkNotNull(value); - if (this.hbaseProperties == null) { - this.hbaseProperties = new HashMap<>(); - } - hbaseProperties.put(key, value); - return this; - } - - @Override - protected Map toConnectorProperties() { - if (this.hbaseProperties != null) { - this.hbaseProperties.forEach( - (key, value) -> { - if (!properties.containsKey(CONNECTOR_PROPERTIES + '.' + key)) { - properties.putString(CONNECTOR_PROPERTIES + '.' + key, value); - } - }); - } - return properties.asMap(); - } -} diff --git a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/example/HBaseFlinkTestConstants.java b/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/example/HBaseFlinkTestConstants.java deleted file mode 100644 index 72565b651e698..0000000000000 --- a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/example/HBaseFlinkTestConstants.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.example; - -import org.apache.hadoop.hbase.util.Bytes; - -/** Constants used during the hbase connector tests. */ -public class HBaseFlinkTestConstants { - - public static final byte[] CF_SOME = Bytes.toBytes("someCf"); - public static final byte[] Q_SOME = Bytes.toBytes("someQual"); - public static final String TEST_TABLE_NAME = "test-table"; - public static final String TMP_DIR = "/tmp/test"; -} diff --git a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/PlannerType.java b/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/PlannerType.java deleted file mode 100644 index 5d302c641ef26..0000000000000 --- a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/PlannerType.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.util; - -/** Planner type to use. */ -public enum PlannerType { - BLINK_PLANNER, - OLD_PLANNER -} diff --git a/flink-python/pyflink/table/descriptors.py b/flink-python/pyflink/table/descriptors.py index 0e7dc287b9086..22c1ccb636aa3 100644 --- a/flink-python/pyflink/table/descriptors.py +++ b/flink-python/pyflink/table/descriptors.py @@ -33,7 +33,6 @@ 'FileSystem', 'Kafka', 'Elasticsearch', - 'HBase', 'Csv', 'Avro', 'Json', @@ -1525,156 +1524,6 @@ def connection_path_prefix(self, path_prefix: str) -> 'Elasticsearch': return self -class HBase(ConnectorDescriptor): - """ - Connector descriptor for Apache HBase. - - .. versionadded:: 1.11.0 - """ - - def __init__(self, version=None, table_name=None, zookeeper_quorum=None, - zookeeper_node_parent=None, write_buffer_flush_max_size=None, - write_buffer_flush_max_rows=None, write_buffer_flush_interval=None): - """ - Constructor of HBase descriptor. - - :param version: HBase version. E.g., "1.4.3". - :param table_name: Name of HBase table. E.g., "testNamespace:testTable", "testDefaultTable" - :param zookeeper_quorum: zookeeper quorum address to connect the HBase cluster. E.g., - "localhost:2181,localhost:2182,localhost:2183" - :param zookeeper_node_parent: zookeeper node path of hbase cluster. E.g, - "/hbase/example-root-znode". - :param write_buffer_flush_max_size: the maximum size. - :param write_buffer_flush_max_rows: number of added rows when begin the request flushing. - :param write_buffer_flush_interval: flush interval. The string should be in format - "{length value}{time unit label}" E.g, "123ms", "1 s", - if not time unit label is specified, it will be - considered as milliseconds. - """ - gateway = get_gateway() - self._j_hbase = gateway.jvm.HBase() - super(HBase, self).__init__(self._j_hbase) - - if version is not None: - self.version(version) - - if table_name is not None: - self.table_name(table_name) - - if zookeeper_quorum is not None: - self.zookeeper_quorum(zookeeper_quorum) - - if zookeeper_node_parent is not None: - self.zookeeper_node_parent(zookeeper_node_parent) - - if write_buffer_flush_max_size is not None: - self.write_buffer_flush_max_size(write_buffer_flush_max_size) - - if write_buffer_flush_max_rows is not None: - self.write_buffer_flush_max_rows(write_buffer_flush_max_rows) - - if write_buffer_flush_interval is not None: - self.write_buffer_flush_interval(write_buffer_flush_interval) - - def version(self, version: str) -> 'HBase': - """ - Set the Apache HBase version to be used, Required. - - :param version: HBase version. E.g., "1.4.3". - :return: This object. - - .. versionadded:: 1.11.0 - """ - if not isinstance(version, str): - version = str(version) - self._j_hbase = self._j_hbase.version(version) - return self - - def table_name(self, table_name: str) -> 'HBase': - """ - Set the HBase table name, Required. - - :param table_name: Name of HBase table. E.g., "testNamespace:testTable", "testDefaultTable" - :return: This object. - - .. versionadded:: 1.11.0 - """ - self._j_hbase = self._j_hbase.tableName(table_name) - return self - - def zookeeper_quorum(self, zookeeper_quorum: str) -> 'HBase': - """ - Set the zookeeper quorum address to connect the HBase cluster, Required. - - :param zookeeper_quorum: zookeeper quorum address to connect the HBase cluster. E.g., - "localhost:2181,localhost:2182,localhost:2183" - :return: This object. - - .. versionadded:: 1.11.0 - """ - self._j_hbase = self._j_hbase.zookeeperQuorum(zookeeper_quorum) - return self - - def zookeeper_node_parent(self, zookeeper_node_parent: str) -> 'HBase': - """ - Set the zookeeper node parent path of HBase cluster. Default to use "/hbase", Optional. - - :param zookeeper_node_parent: zookeeper node path of hbase cluster. E.g, - "/hbase/example-root-znode". - :return: This object - - .. versionadded:: 1.11.0 - """ - self._j_hbase = self._j_hbase.zookeeperNodeParent(zookeeper_node_parent) - return self - - def write_buffer_flush_max_size(self, max_size: Union[int, str]) -> 'HBase': - """ - Set threshold when to flush buffered request based on the memory byte size of rows currently - added. - - :param max_size: the maximum size. - :return: This object. - - .. versionadded:: 1.11.0 - """ - if not isinstance(max_size, str): - max_size = str(max_size) - self._j_hbase = self._j_hbase.writeBufferFlushMaxSize(max_size) - return self - - def write_buffer_flush_max_rows(self, write_buffer_flush_max_rows: int) -> 'HBase': - """ - Set threshold when to flush buffered request based on the number of rows currently added. - Defaults to not set, i.e. won;t flush based on the number of buffered rows, Optional. - - :param write_buffer_flush_max_rows: number of added rows when begin the request flushing. - :return: This object. - - .. versionadded:: 1.11.0 - """ - self._j_hbase = self._j_hbase.writeBufferFlushMaxRows(write_buffer_flush_max_rows) - return self - - def write_buffer_flush_interval(self, interval: Union[str, int]) -> 'HBase': - """ - Set an interval when to flushing buffered requesting if the interval passes, in - milliseconds. - Defaults to not set, i.e. won't flush based on flush interval, Optional. - - :param interval: flush interval. The string should be in format - "{length value}{time unit label}" E.g, "123ms", "1 s", if not time unit - label is specified, it will be considered as milliseconds. - :return: This object. - - .. versionadded:: 1.11.0 - """ - if not isinstance(interval, str): - interval = str(interval) - self._j_hbase = self._j_hbase.writeBufferFlushInterval(interval) - return self - - class CustomConnectorDescriptor(ConnectorDescriptor): """ Describes a custom connector to an other system. diff --git a/flink-python/pyflink/table/tests/test_descriptor.py b/flink-python/pyflink/table/tests/test_descriptor.py index d08ff090f6983..92bada329503a 100644 --- a/flink-python/pyflink/table/tests/test_descriptor.py +++ b/flink-python/pyflink/table/tests/test_descriptor.py @@ -22,7 +22,7 @@ from pyflink.java_gateway import get_gateway from pyflink.table.descriptors import (FileSystem, OldCsv, Rowtime, Schema, Kafka, Elasticsearch, Csv, Avro, Json, CustomConnectorDescriptor, - CustomFormatDescriptor, HBase) + CustomFormatDescriptor) from pyflink.table.table_schema import TableSchema from pyflink.table.types import DataTypes from pyflink.testing.test_case_utils import (PyFlinkTestCase, PyFlinkOldStreamTableTestCase, @@ -393,114 +393,6 @@ def test_custom_connector(self): self.assertEqual(expected, properties) -class HBaseDescriptorTests(PyFlinkTestCase): - - @classmethod - def setUpClass(cls): - super(HBaseDescriptorTests, cls).setUpClass() - cls._cxt_clz_loader = get_gateway().jvm.Thread.currentThread().getContextClassLoader() - _load_specific_flink_module_jars('/flink-connectors/flink-connector-hbase-base') - - def test_version(self): - hbase = HBase().version("1.4.3") - - properties = hbase.to_properties() - expected = {'connector.version': '1.4.3', - 'connector.type': 'hbase', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - hbase = HBase().version(1.1) - properties = hbase.to_properties() - expected = {'connector.version': '1.1', - 'connector.type': 'hbase', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - def test_table_name(self): - hbase = HBase().table_name('tableName1') - - properties = hbase.to_properties() - expected = {'connector.type': 'hbase', - 'connector.table-name': 'tableName1', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - def test_zookeeper_quorum(self): - hbase = HBase().zookeeper_quorum("localhost:2181,localhost:2182") - - properties = hbase.to_properties() - expected = {'connector.type': 'hbase', - 'connector.zookeeper.quorum': 'localhost:2181,localhost:2182', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - def test_zookeeper_node_parent(self): - hbase = HBase().zookeeper_node_parent('/hbase/example-root-znode') - - properties = hbase.to_properties() - expected = {'connector.type': 'hbase', - 'connector.zookeeper.znode.parent': '/hbase/example-root-znode', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - def test_write_buffer_flush_max_size(self): - hbase = HBase().write_buffer_flush_max_size('1000') - - properties = hbase.to_properties() - expected = {'connector.type': 'hbase', - 'connector.write.buffer-flush.max-size': '1000 bytes', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - hbase = HBase().write_buffer_flush_max_size(1000) - properties = hbase.to_properties() - self.assertEqual(expected, properties) - - hbase = HBase().write_buffer_flush_max_size('10mb') - properties = hbase.to_properties() - expected = {'connector.type': 'hbase', - 'connector.write.buffer-flush.max-size': '10 mb', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - def test_write_buffer_flush_max_rows(self): - hbase = HBase().write_buffer_flush_max_rows(10) - - properties = hbase.to_properties() - expected = {'connector.type': 'hbase', - 'connector.write.buffer-flush.max-rows': '10', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - def test_write_buffer_flush_interval(self): - hbase = HBase().write_buffer_flush_interval('123') - - properties = hbase.to_properties() - expected = {'connector.type': 'hbase', - 'connector.write.buffer-flush.interval': '123', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - hbase = HBase().write_buffer_flush_interval(123) - - properties = hbase.to_properties() - self.assertEqual(expected, properties) - - hbase = HBase().write_buffer_flush_interval('123ms') - - properties = hbase.to_properties() - expected = {'connector.type': 'hbase', - 'connector.write.buffer-flush.interval': '123ms', - 'connector.property-version': '1'} - self.assertEqual(expected, properties) - - @classmethod - def tearDownClass(cls): - if cls._cxt_clz_loader is not None: - get_gateway().jvm.Thread.currentThread().setContextClassLoader(cls._cxt_clz_loader) - - class OldCsvDescriptorTests(PyFlinkTestCase): def test_field_delimiter(self): From 523edc400ac70cbcbe62b22449e2d7cbbbf1f270 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 18 May 2021 09:53:19 +0200 Subject: [PATCH 2/2] [hotfix][hbase] Fix warnings around decimals in HBaseTestBase --- .../connector/hbase1/util/HBaseTestBase.java | 16 ++++++++-------- .../connector/hbase2/util/HBaseTestBase.java | 16 ++++++++-------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java index 4d267d3043d83..1a3dee1456757 100644 --- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java +++ b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java @@ -120,7 +120,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-18 19:00:00"), Date.valueOf("2019-08-18"), Time.valueOf("19:00:00"), - new BigDecimal(12345678.0001))); + new BigDecimal("12345678.0001"))); puts.add( putRow( 2, @@ -133,7 +133,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-18 19:01:00"), Date.valueOf("2019-08-18"), Time.valueOf("19:01:00"), - new BigDecimal(12345678.0002))); + new BigDecimal("12345678.0002"))); puts.add( putRow( 3, @@ -146,7 +146,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-18 19:02:00"), Date.valueOf("2019-08-18"), Time.valueOf("19:02:00"), - new BigDecimal(12345678.0003))); + new BigDecimal("12345678.0003"))); puts.add( putRow( 4, @@ -159,7 +159,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-18 19:03:00"), Date.valueOf("2019-08-18"), Time.valueOf("19:03:00"), - new BigDecimal(12345678.0004))); + new BigDecimal("12345678.0004"))); puts.add( putRow( 5, @@ -172,7 +172,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-19 19:10:00"), Date.valueOf("2019-08-19"), Time.valueOf("19:10:00"), - new BigDecimal(12345678.0005))); + new BigDecimal("12345678.0005"))); puts.add( putRow( 6, @@ -185,7 +185,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-19 19:20:00"), Date.valueOf("2019-08-19"), Time.valueOf("19:20:00"), - new BigDecimal(12345678.0006))); + new BigDecimal("12345678.0006"))); puts.add( putRow( 7, @@ -198,7 +198,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-19 19:30:00"), Date.valueOf("2019-08-19"), Time.valueOf("19:30:00"), - new BigDecimal(12345678.0007))); + new BigDecimal("12345678.0007"))); puts.add( putRow( 8, @@ -211,7 +211,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-19 19:40:00"), Date.valueOf("2019-08-19"), Time.valueOf("19:40:00"), - new BigDecimal(12345678.0008))); + new BigDecimal("12345678.0008"))); // append rows to table table.put(puts); diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java index 1fbf11b8a2578..097042ee2ce68 100644 --- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java +++ b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java @@ -120,7 +120,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-18 19:00:00"), Date.valueOf("2019-08-18"), Time.valueOf("19:00:00"), - new BigDecimal(12345678.0001))); + new BigDecimal("12345678.0001"))); puts.add( putRow( 2, @@ -133,7 +133,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-18 19:01:00"), Date.valueOf("2019-08-18"), Time.valueOf("19:01:00"), - new BigDecimal(12345678.0002))); + new BigDecimal("12345678.0002"))); puts.add( putRow( 3, @@ -146,7 +146,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-18 19:02:00"), Date.valueOf("2019-08-18"), Time.valueOf("19:02:00"), - new BigDecimal(12345678.0003))); + new BigDecimal("12345678.0003"))); puts.add( putRow( 4, @@ -159,7 +159,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-18 19:03:00"), Date.valueOf("2019-08-18"), Time.valueOf("19:03:00"), - new BigDecimal(12345678.0004))); + new BigDecimal("12345678.0004"))); puts.add( putRow( 5, @@ -172,7 +172,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-19 19:10:00"), Date.valueOf("2019-08-19"), Time.valueOf("19:10:00"), - new BigDecimal(12345678.0005))); + new BigDecimal("12345678.0005"))); puts.add( putRow( 6, @@ -185,7 +185,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-19 19:20:00"), Date.valueOf("2019-08-19"), Time.valueOf("19:20:00"), - new BigDecimal(12345678.0006))); + new BigDecimal("12345678.0006"))); puts.add( putRow( 7, @@ -198,7 +198,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-19 19:30:00"), Date.valueOf("2019-08-19"), Time.valueOf("19:30:00"), - new BigDecimal(12345678.0007))); + new BigDecimal("12345678.0007"))); puts.add( putRow( 8, @@ -211,7 +211,7 @@ private static void createHBaseTable1() throws IOException { Timestamp.valueOf("2019-08-19 19:40:00"), Date.valueOf("2019-08-19"), Time.valueOf("19:40:00"), - new BigDecimal(12345678.0008))); + new BigDecimal("12345678.0008"))); // append rows to table table.put(puts);