From 1a42c6226faba2d00880f7a3b91f2f4e9f6b0267 Mon Sep 17 00:00:00 2001 From: liuliu Date: Sat, 2 Jul 2022 17:46:45 +0800 Subject: [PATCH 1/3] [feat][hive3]add hive3 connector --- .../chunjun-connector-ftp/pom.xml | 7 + .../chunjun-connector-hive3/pom.xml | 312 +++++++++ .../connector/hive3/conf/HdfsConf.java | 217 +++++++ .../converter/HdfsOrcColumnConverter.java | 239 +++++++ .../hive3/converter/HdfsOrcRowConverter.java | 199 ++++++ .../converter/HdfsParquetColumnConverter.java | 251 ++++++++ .../converter/HdfsParquetRowConverter.java | 261 ++++++++ .../hive3/converter/HdfsRawTypeConverter.java | 91 +++ .../converter/HdfsTextColumnConverter.java | 223 +++++++ .../hive3/converter/HdfsTextRowConverter.java | 241 +++++++ .../connector/hive3/enums/CompressType.java | 95 +++ .../connector/hive3/enums/FileType.java | 51 ++ .../hive3/inputSplit/HdfsOrcInputSplit.java | 65 ++ .../inputSplit/HdfsParquetInputSplit.java | 44 ++ .../hive3/inputSplit/HdfsTextInputSplit.java | 62 ++ .../hive3/sink/BaseHdfsOutputFormat.java | 276 ++++++++ .../hive3/sink/HdfsOrcOutputFormat.java | 239 +++++++ .../hive3/sink/HdfsParquetOutputFormat.java | 276 ++++++++ .../hive3/sink/HdfsTextOutputFormat.java | 150 +++++ .../sink/HdfsTransactionOutputFormat.java | 360 +++++++++++ .../hive3/sink/Hive3OutputFormatBuilder.java | 58 ++ .../hive3/sink/Hive3SinkFactory.java | 114 ++++ .../hive3/source/BaseHdfsInputFormat.java | 155 +++++ .../hive3/source/HdfsOrcInputFormat.java | 260 ++++++++ .../hive3/source/HdfsParquetInputFormat.java | 393 ++++++++++++ .../hive3/source/HdfsPathFilter.java | 72 +++ .../hive3/source/HdfsTextInputFormat.java | 153 +++++ .../source/HdfsTransactionInputFormat.java | 207 ++++++ .../hive3/source/Hive3InputFormatBuilder.java | 59 ++ .../hive3/source/Hive3SourceFactory.java | 69 ++ .../connector/hive3/util/Hive3Util.java | 601 ++++++++++++++++++ chunjun-connectors/pom.xml | 1 + pom.xml | 22 +- 33 files changed, 5819 insertions(+), 4 deletions(-) create mode 100644 chunjun-connectors/chunjun-connector-hive3/pom.xml create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/conf/HdfsConf.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsOrcColumnConverter.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsOrcRowConverter.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsParquetColumnConverter.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsParquetRowConverter.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsRawTypeConverter.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsTextColumnConverter.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsTextRowConverter.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/enums/CompressType.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/enums/FileType.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsOrcInputSplit.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsParquetInputSplit.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsTextInputSplit.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/BaseHdfsOutputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsOrcOutputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsParquetOutputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsTextOutputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsTransactionOutputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/Hive3OutputFormatBuilder.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/Hive3SinkFactory.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/BaseHdfsInputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsOrcInputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsParquetInputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsPathFilter.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsTextInputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsTransactionInputFormat.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3InputFormatBuilder.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3SourceFactory.java create mode 100644 chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/util/Hive3Util.java diff --git a/chunjun-connectors/chunjun-connector-ftp/pom.xml b/chunjun-connectors/chunjun-connector-ftp/pom.xml index 52fd8bf381..2a7ad2f201 100644 --- a/chunjun-connectors/chunjun-connector-ftp/pom.xml +++ b/chunjun-connectors/chunjun-connector-ftp/pom.xml @@ -62,6 +62,13 @@ under the License. easyexcel 3.0.1 + + + org.mortbay.jetty + jetty-util + 6.1.26 + provided + diff --git a/chunjun-connectors/chunjun-connector-hive3/pom.xml b/chunjun-connectors/chunjun-connector-hive3/pom.xml new file mode 100644 index 0000000000..00debe640b --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/pom.xml @@ -0,0 +1,312 @@ + + + + chunjun-connectors + com.dtstack.chunjun + 1.12-SNAPSHOT + + 4.0.0 + + chunjun-connector-hive3 + ChunJun : Connectors : Hive3 + + + 8 + 8 + 3.1.3 + 3.1.4 + + + + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop3.version} + + + protobuf-java + com.google.protobuf + + + com.google.protobuf + protobuf-java-util + + + commons-cli + commons-cli + + + + + + org.apache.hadoop + hadoop-common + ${hadoop3.version} + + + protobuf-java + com.google.protobuf + + + com.google.protobuf + protobuf-java-util + + + + + + org.apache.hadoop + hadoop-yarn-common + ${hadoop3.version} + + + + org.apache.hadoop + hadoop-yarn-client + ${hadoop3.version} + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop3.version} + + + org.apache.hadoop + hadoop-yarn-api + ${hadoop3.version} + + + + + org.apache.hive + hive-exec + ${hive.version} + + + + org.apache.hive + hive-serde + ${hive.version} + + + org.apache.hadoop + hadoop-yarn-api + + + org.xerial.snappy + snappy-java + + + + + + org.apache.hive + hive-streaming + ${hive.version} + + + + org.apache.thrift + libfb303 + 0.9.3 + + + org.apache.calcite + calcite-core + 1.19.0.7.1.3.0-100 + + + parquet-hadoop + org.apache.parquet + 1.8.3 + + + org.xerial.snappy + snappy-java + + + + + + org.xerial.snappy + snappy-java + 1.1.4 + + + + httpcore + org.apache.httpcomponents + 4.4.5 + + + + httpclient + org.apache.httpcomponents + 4.5.2 + + + + org.apache.hive + hive-metastore + ${hive.version} + + + org.datanucleus + datanucleus-core + + + org.datanucleus + datanucleus-rdbms + + + + + + org.apache.hive + hive-common + ${hive.version} + + + + org.datanucleus + datanucleus-core + 5.0.1 + + + + org.datanucleus + datanucleus-api-jdo + 5.0.1 + + + + org.datanucleus + datanucleus-rdbms + 5.0.1 + + + + org.apache.commons + commons-collections4 + 4.3 + + + + com.google.guava + guava + 27.0-jre + + + + com.dtstack.chunjun + chunjun-connector-stream + 1.12-SNAPSHOT + + + + commons-cli + commons-cli + 1.3.1 + provided + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.hive3.com.google.common + + + com.google.guava + shade.hive3.com.google.guava + + + org.checkerframework + shade.hive3.org.checkerframework + + + com.google.thirdparty + shade.hive3.com.google.thirdparty + + + org.apache.hadoop.util.ShutdownHookManager + shade.hive3.org.apache.hadoop.util.ShutdownHookManager + + + + + META-INF/services/java.sql.Driver + + + META-INF/services + java.sql.hive2.Driver + + + + + + + + + maven-antrun-plugin + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/conf/HdfsConf.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/conf/HdfsConf.java new file mode 100644 index 0000000000..1e6bd7dba1 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/conf/HdfsConf.java @@ -0,0 +1,217 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.conf; + +import com.dtstack.chunjun.conf.BaseFileConf; + +import org.apache.parquet.hadoop.ParquetWriter; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** @author liuliu 2022/3/23 */ +public class HdfsConf extends BaseFileConf { + private String defaultFS; + private String fileType; + /** hadoop高可用相关配置 * */ + private Map hadoopConfig = new HashMap<>(16); + + private String filterRegex = ""; + private String fieldDelimiter = "\001"; + private int rowGroupSize = ParquetWriter.DEFAULT_BLOCK_SIZE; + private boolean enableDictionary = true; + private List fullColumnName; + private List fullColumnType; + private int[] fullColumnIndexes; + + /** hive3 事务表 */ + private boolean hiveTransactionTable = false; + + private boolean isTransaction = false; + + private String partitionName; + private String schema; + private String table; + + public String getDefaultFS() { + return defaultFS; + } + + public void setDefaultFS(String defaultFS) { + this.defaultFS = defaultFS; + } + + public String getFileType() { + return fileType; + } + + public void setFileType(String fileType) { + this.fileType = fileType; + } + + public Map getHadoopConfig() { + return hadoopConfig; + } + + public void setHadoopConfig(Map hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } + + public String getFilterRegex() { + return filterRegex; + } + + public void setFilterRegex(String filterRegex) { + this.filterRegex = filterRegex; + } + + public String getFieldDelimiter() { + return fieldDelimiter; + } + + public void setFieldDelimiter(String fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + } + + public int getRowGroupSize() { + return rowGroupSize; + } + + public void setRowGroupSize(int rowGroupSize) { + this.rowGroupSize = rowGroupSize; + } + + public boolean isEnableDictionary() { + return enableDictionary; + } + + public void setEnableDictionary(boolean enableDictionary) { + this.enableDictionary = enableDictionary; + } + + public List getFullColumnName() { + return fullColumnName; + } + + public void setFullColumnName(List fullColumnName) { + this.fullColumnName = fullColumnName; + } + + public List getFullColumnType() { + return fullColumnType; + } + + public void setFullColumnType(List fullColumnType) { + this.fullColumnType = fullColumnType; + } + + public int[] getFullColumnIndexes() { + return fullColumnIndexes; + } + + public void setFullColumnIndexes(int[] fullColumnIndexes) { + this.fullColumnIndexes = fullColumnIndexes; + } + + public boolean isHiveTransactionTable() { + return hiveTransactionTable; + } + + public void setHiveTransactionTable(boolean hiveTransactionTable) { + this.hiveTransactionTable = hiveTransactionTable; + } + + public String getPartitionName() { + return partitionName; + } + + public void setPartitionName(String partitionName) { + this.partitionName = partitionName; + } + + public String getSchema() { + return schema; + } + + public void setSchema(String schema) { + this.schema = schema; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public boolean isTransaction() { + return isTransaction; + } + + public void setTransaction(boolean transaction) { + isTransaction = transaction; + } + + @Override + public String toString() { + return "HdfsConf{" + + "defaultFS='" + + defaultFS + + '\'' + + ", fileType='" + + fileType + + '\'' + + ", hadoopConfig=" + + hadoopConfig + + ", filterRegex='" + + filterRegex + + '\'' + + ", fieldDelimiter='" + + fieldDelimiter + + '\'' + + ", rowGroupSize=" + + rowGroupSize + + ", enableDictionary=" + + enableDictionary + + ", fullColumnName=" + + fullColumnName + + ", fullColumnType=" + + fullColumnType + + ", fullColumnIndexes=" + + Arrays.toString(fullColumnIndexes) + + ", hiveTransactionTable=" + + hiveTransactionTable + + ", isTransaction=" + + isTransaction + + ", partitionName='" + + partitionName + + '\'' + + ", schema='" + + schema + + '\'' + + ", table='" + + table + + '\'' + + '}' + + super.toString(); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsOrcColumnConverter.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsOrcColumnConverter.java new file mode 100644 index 0000000000..7322470604 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsOrcColumnConverter.java @@ -0,0 +1,239 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.converter; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.IDeserializationConverter; +import com.dtstack.chunjun.converter.ISerializationConverter; +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.ColumnRowData; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.ByteColumn; +import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.SqlDateColumn; +import com.dtstack.chunjun.element.column.StringColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; +import com.dtstack.chunjun.throwable.WriteRecordException; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.hadoop.hive.common.type.Date; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.io.BytesWritable; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.List; + +/** @author liuliu 2022/3/23 */ +public class HdfsOrcColumnConverter + extends AbstractRowConverter, LogicalType> { + + HdfsConf hdfsConf; + + public HdfsOrcColumnConverter(RowType rowType, HdfsConf hdfsConf) { + super(rowType, hdfsConf); + this.hdfsConf = hdfsConf; + for (int i = 0; i < rowType.getFieldCount(); i++) { + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(rowType.getTypeAt(i)), rowType.getTypeAt(i))); + } + } + + @Override + @SuppressWarnings("unchecked") + protected ISerializationConverter> wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, LogicalType type) { + return (rowData, index, data) -> { + if (rowData == null || rowData.isNullAt(index)) { + data.add(null); + } else { + serializationConverter.serialize(rowData, index, data); + } + }; + } + + @Override + @SuppressWarnings("unchecked") + public RowData toInternal(RowData input) throws Exception { + ColumnRowData row = new ColumnRowData(input.getArity()); + if (input instanceof GenericRowData) { + List fieldConfList = commonConf.getColumn(); + GenericRowData genericRowData = (GenericRowData) input; + for (int i = 0; i < fieldConfList.size(); i++) { + row.addField( + assembleFieldProps( + fieldConfList.get(i), + (AbstractBaseColumn) + toInternalConverters + .get(i) + .deserialize(genericRowData.getField(i)))); + } + } else { + throw new ChunJunRuntimeException( + "Error RowData type, RowData:[" + + input + + "] should be instance of GenericRowData."); + } + return row; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); + } + + @Override + public List toExternal(RowData rowData, List output) throws Exception { + for (int index = 0; index < hdfsConf.getFullColumnName().size(); index++) { + int columnIndex = hdfsConf.getFullColumnIndexes()[index]; + if (columnIndex == -1) { + output.add(null); + continue; + } + toExternalConverters.get(columnIndex).serialize(rowData, columnIndex, output); + } + return output; + } + + @Override + @SuppressWarnings("all") + protected IDeserializationConverter createInternalConverter(LogicalType logicalType) { + switch (logicalType.getTypeRoot()) { + case BOOLEAN: + return (IDeserializationConverter) BooleanColumn::new; + case TINYINT: + return (IDeserializationConverter) ByteColumn::new; + case SMALLINT: + case INTEGER: + return (IDeserializationConverter) + BigDecimalColumn::new; + case BIGINT: + return (IDeserializationConverter) BigDecimalColumn::new; + case FLOAT: + return (IDeserializationConverter) BigDecimalColumn::new; + case DOUBLE: + return (IDeserializationConverter) + BigDecimalColumn::new; + case DECIMAL: + return (IDeserializationConverter) + BigDecimalColumn::new; + case VARCHAR: + return (IDeserializationConverter) StringColumn::new; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (IDeserializationConverter< + org.apache.hadoop.hive.common.type.Timestamp, AbstractBaseColumn>) + val -> { + Instant instant = Instant.ofEpochMilli(val.toEpochMilli()); + LocalDateTime localDateTime = + LocalDateTime.ofInstant(instant, ZoneId.of("UTC")); + return new TimestampColumn(Timestamp.valueOf(localDateTime)); + }; + case DATE: + return (IDeserializationConverter) + val -> new SqlDateColumn(val.toEpochDay()); + case BINARY: + return (IDeserializationConverter) BytesColumn::new; + default: + throw new UnsupportedTypeException(logicalType); + } + } + + @Override + protected ISerializationConverter> createExternalConverter( + LogicalType logicalType) { + + switch (logicalType.getTypeRoot()) { + case BOOLEAN: + return (rowData, index, list) -> + list.add(((ColumnRowData) rowData).getField(index).asBoolean()); + case TINYINT: + return (rowData, index, list) -> list.add(rowData.getByte(index)); + case SMALLINT: + case INTEGER: + return (rowData, index, list) -> + list.add(((ColumnRowData) rowData).getField(index).asYearInt()); + case BIGINT: + return (rowData, index, list) -> + list.add(((ColumnRowData) rowData).getField(index).asLong()); + case FLOAT: + return (rowData, index, list) -> + list.add(((ColumnRowData) rowData).getField(index).asFloat()); + case DOUBLE: + return (rowData, index, list) -> + list.add(((ColumnRowData) rowData).getField(index).asDouble()); + case DECIMAL: + return (rowData, index, list) -> { + DecimalType decimalType = (DecimalType) logicalType; + int precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + HiveDecimal hiveDecimal = + HiveDecimal.create( + rowData.getDecimal(index, precision, scale).toBigDecimal()); + hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, precision, scale); + if (hiveDecimal == null) { + String msg = + String.format( + "The [%s] data data [%s] precision and scale do not match the metadata:decimal(%s, %s)", + index, precision, scale, rowData); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + list.add(new HiveDecimalWritable(hiveDecimal)); + }; + case VARCHAR: + return (rowData, index, list) -> + list.add(((ColumnRowData) rowData).getField(index).asString()); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, list) -> { + Timestamp timestamp = ((ColumnRowData) rowData).getField(index).asTimestamp(); + list.add( + org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli( + timestamp.getTime(), timestamp.getNanos())); + }; + case DATE: + return (rowData, index, list) -> { + TimestampData timestamp = rowData.getTimestamp(index, 6); + list.add(Date.ofEpochMilli(timestamp.getMillisecond())); + }; + case BINARY: + return (rowData, index, list) -> + list.add(new BytesWritable(rowData.getBinary(index))); + default: + throw new UnsupportedTypeException(logicalType); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsOrcRowConverter.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsOrcRowConverter.java new file mode 100644 index 0000000000..5af8aebe93 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsOrcRowConverter.java @@ -0,0 +1,199 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.converter; + +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.IDeserializationConverter; +import com.dtstack.chunjun.converter.ISerializationConverter; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; + +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.io.BytesWritable; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; + +/** @author liuliu 2022/3/23 */ +public class HdfsOrcRowConverter + extends AbstractRowConverter, LogicalType> { + public HdfsOrcRowConverter(RowType rowType, HdfsConf hdfsConf) { + super(rowType, hdfsConf); + for (int i = 0; i < rowType.getFieldCount(); i++) { + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(fieldTypes[i]), fieldTypes[i])); + } + } + + @Override + public RowData toInternal(RowData input) throws Exception { + GenericRowData row = new GenericRowData(input.getArity()); + if (input instanceof GenericRowData) { + GenericRowData genericRowData = (GenericRowData) input; + for (int i = 0; i < input.getArity(); i++) { + row.setField( + i, toInternalConverters.get(i).deserialize(genericRowData.getField(i))); + } + } else { + throw new ChunJunRuntimeException( + "Error RowData type, RowData:[" + + input + + "] should be instance of GenericRowData."); + } + return row; + } + + @Override + public List toExternal(RowData rowData, List data) throws Exception { + for (int index = 0; index < rowData.getArity(); index++) { + toExternalConverters.get(index).serialize(rowData, index, data); + } + return data; + } + + @Override + @SuppressWarnings("all") + protected IDeserializationConverter createInternalConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return val -> null; + case BOOLEAN: + return (IDeserializationConverter) val -> val; + case TINYINT: + return (IDeserializationConverter) val -> val; + case SMALLINT: + return (IDeserializationConverter) val -> val; + case INTEGER: + return (IDeserializationConverter) val -> val; + case BIGINT: + return (IDeserializationConverter) val -> val; + case DATE: + return (IDeserializationConverter) + val -> (int) val.toLocalDate().toEpochDay(); + case FLOAT: + return (IDeserializationConverter) val -> val; + case DOUBLE: + return (IDeserializationConverter) val -> val; + case CHAR: + case VARCHAR: + return (IDeserializationConverter) StringData::fromString; + case DECIMAL: + return (IDeserializationConverter) + val -> DecimalData.fromBigDecimal(val, val.precision(), val.scale()); + case BINARY: + case VARBINARY: + return (IDeserializationConverter) val -> val; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (IDeserializationConverter) + TimestampData::fromTimestamp; + case INTERVAL_DAY_TIME: + case INTERVAL_YEAR_MONTH: + case ARRAY: + case MAP: + case MULTISET: + case ROW: + case RAW: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + default: + throw new UnsupportedTypeException(type); + } + } + + @Override + protected ISerializationConverter> createExternalConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return (rowData, index, data) -> data.add(null); + case BOOLEAN: + return (rowData, index, data) -> data.add(rowData.getBoolean(index)); + case TINYINT: + return (rowData, index, data) -> data.add(rowData.getByte(index)); + case SMALLINT: + return (rowData, index, data) -> data.add(rowData.getShort(index)); + case INTEGER: + return (rowData, index, data) -> data.add(rowData.getInt(index)); + case BIGINT: + return (rowData, index, data) -> data.add(rowData.getLong(index)); + case DATE: + return (rowData, index, data) -> + data.add( + org.apache.hadoop.hive.common.type.Date.ofEpochDay( + rowData.getInt(index))); + case FLOAT: + return (rowData, index, data) -> data.add(rowData.getFloat(index)); + case DOUBLE: + return (rowData, index, data) -> data.add(rowData.getDouble(index)); + case CHAR: + case VARCHAR: + return (rowData, index, data) -> data.add(rowData.getString(index).toString()); + case DECIMAL: + return (rowData, index, data) -> { + int precision = ((DecimalType) type).getPrecision(); + int scale = ((DecimalType) type).getScale(); + HiveDecimal hiveDecimal = + HiveDecimal.create( + rowData.getDecimal(index, precision, scale).toBigDecimal()); + hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, precision, scale); + data.add(new HiveDecimalWritable(hiveDecimal)); + }; + case BINARY: + return (rowData, index, data) -> + data.add(new BytesWritable(rowData.getBinary(index))); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, data) -> { + TimestampData timestampData = + rowData.getTimestamp(index, ((TimestampType) type).getPrecision()); + data.add( + org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli( + timestampData.getMillisecond(), + timestampData.getNanoOfMillisecond())); + }; + case INTERVAL_DAY_TIME: + case INTERVAL_YEAR_MONTH: + case ARRAY: + case MAP: + case MULTISET: + case ROW: + case RAW: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + default: + throw new UnsupportedTypeException(type); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsParquetColumnConverter.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsParquetColumnConverter.java new file mode 100644 index 0000000000..3e2233b7f6 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsParquetColumnConverter.java @@ -0,0 +1,251 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.converter; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.IDeserializationConverter; +import com.dtstack.chunjun.converter.ISerializationConverter; +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.ColumnRowData; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.ByteColumn; +import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.StringColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; +import com.dtstack.chunjun.throwable.WriteRecordException; +import com.dtstack.chunjun.util.DateUtil; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.io.api.Binary; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import java.util.stream.Collectors; + +/** @author liuliu 2022/3/23 */ +public class HdfsParquetColumnConverter + extends AbstractRowConverter { + private final List columnNameList; + HdfsConf hdfsConf; + + public HdfsParquetColumnConverter(RowType rowType, HdfsConf hdfsConf) { + super(rowType, hdfsConf); + this.hdfsConf = hdfsConf; + List fieldConfList = hdfsConf.getColumn(); + columnNameList = + fieldConfList.stream().map(FieldConf::getName).collect(Collectors.toList()); + for (int i = 0; i < rowType.getFieldCount(); i++) { + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(fieldTypes[i]), fieldTypes[i])); + } + } + + @Override + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, LogicalType type) { + return (rowData, index, group) -> { + if (rowData == null || rowData.isNullAt(index)) { + // do nothing + } else { + serializationConverter.serialize(rowData, index, group); + } + }; + } + + @Override + @SuppressWarnings("unchecked") + public RowData toInternal(RowData input) throws Exception { + ColumnRowData row = new ColumnRowData(input.getArity()); + if (input instanceof GenericRowData) { + List fieldConfList = commonConf.getColumn(); + GenericRowData genericRowData = (GenericRowData) input; + for (int i = 0; i < fieldConfList.size(); i++) { + row.addField( + assembleFieldProps( + fieldConfList.get(i), + (AbstractBaseColumn) + toInternalConverters + .get(i) + .deserialize(genericRowData.getField(i)))); + } + } else { + throw new ChunJunRuntimeException( + "Error RowData type, RowData:[" + + input + + "] should be instance of GenericRowData."); + } + return row; + } + + @Override + @SuppressWarnings("unchecked") + public Group toExternal(RowData rowData, Group group) throws Exception { + for (int index = 0; index < hdfsConf.getFullColumnName().size(); index++) { + int columnIndex = hdfsConf.getFullColumnIndexes()[index]; + if (columnIndex == -1) { + continue; + } + toExternalConverters.get(columnIndex).serialize(rowData, columnIndex, group); + } + return group; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); + } + + @Override + @SuppressWarnings("all") + protected IDeserializationConverter createInternalConverter(LogicalType logicalType) { + switch (logicalType.getTypeRoot()) { + case BOOLEAN: + return (IDeserializationConverter) BooleanColumn::new; + case TINYINT: + return (IDeserializationConverter) ByteColumn::new; + case SMALLINT: + case INTEGER: + return (IDeserializationConverter) + BigDecimalColumn::new; + case BIGINT: + return (IDeserializationConverter) BigDecimalColumn::new; + case FLOAT: + return (IDeserializationConverter) BigDecimalColumn::new; + case DOUBLE: + return (IDeserializationConverter) + BigDecimalColumn::new; + case DECIMAL: + return (IDeserializationConverter) + BigDecimalColumn::new; + case VARCHAR: + return (IDeserializationConverter) StringColumn::new; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (IDeserializationConverter) + TimestampColumn::new; + case DATE: + return (IDeserializationConverter) + val -> new TimestampColumn(DateUtil.getTimestampFromStr(val)); + case BINARY: + return (IDeserializationConverter) BytesColumn::new; + default: + throw new UnsupportedTypeException(logicalType); + } + } + + @Override + protected ISerializationConverter createExternalConverter(LogicalType logicalType) { + switch (logicalType.getTypeRoot()) { + case BOOLEAN: + return (rowData, index, data) -> + data.add( + columnNameList.get(index), + ((ColumnRowData) rowData).getField(index).asBoolean()); + case TINYINT: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getByte(index)); + case SMALLINT: + case INTEGER: + return (rowData, index, data) -> + data.add( + columnNameList.get(index), + ((ColumnRowData) rowData).getField(index).asYearInt()); + + case BIGINT: + return (rowData, index, data) -> + data.add( + columnNameList.get(index), + ((ColumnRowData) rowData).getField(index).asLong()); + case FLOAT: + return (rowData, index, data) -> + data.add( + columnNameList.get(index), + ((ColumnRowData) rowData).getField(index).asFloat()); + case DOUBLE: + return (rowData, index, data) -> + data.add( + columnNameList.get(index), + ((ColumnRowData) rowData).getField(index).asDouble()); + case DECIMAL: + return (rowData, index, data) -> { + DecimalType decimalType = (DecimalType) logicalType; + int precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + HiveDecimal hiveDecimal = + HiveDecimal.create( + rowData.getDecimal(index, precision, scale).toBigDecimal()); + hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, precision, scale); + if (hiveDecimal == null) { + String msg = + String.format( + "The [%s] data data [%s] precision and scale do not match the metadata:decimal(%s, %s)", + index, precision, scale, rowData); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + data.add( + columnNameList.get(index), + Hive3Util.decimalToBinary(hiveDecimal, precision, scale)); + }; + case VARCHAR: + return (rowData, index, data) -> + data.add( + columnNameList.get(index), + ((ColumnRowData) rowData).getField(index).asString()); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, data) -> { + Timestamp timestamp = ((ColumnRowData) rowData).getField(index).asTimestamp(); + data.add(columnNameList.get(index), Hive3Util.timeToBinary(timestamp)); + }; + case DATE: + return (rowData, index, data) -> { + TimestampData timestampData = rowData.getTimestamp(index, 6); + Date date = Date.valueOf(timestampData.toLocalDateTime().toLocalDate()); + data.add(columnNameList.get(index), DateWritable.dateToDays(date)); + }; + case BINARY: + return (rowData, index, data) -> + data.add( + columnNameList.get(index), + Binary.fromReusedByteArray( + ((ColumnRowData) rowData).getField(index).asBytes())); + default: + throw new UnsupportedTypeException(logicalType); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsParquetRowConverter.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsParquetRowConverter.java new file mode 100644 index 0000000000..06a2d4e99f --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsParquetRowConverter.java @@ -0,0 +1,261 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.converter; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.IDeserializationConverter; +import com.dtstack.chunjun.converter.ISerializationConverter; +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.ColumnRowData; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; +import com.dtstack.chunjun.throwable.WriteRecordException; +import com.dtstack.chunjun.util.DateUtil; + +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.io.api.Binary; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.util.List; +import java.util.stream.Collectors; + +/** @author liuliu 2022/3/23 */ +public class HdfsParquetRowConverter + extends AbstractRowConverter { + + HdfsConf hdfsConf; + private final List columnNameList; + + public HdfsParquetRowConverter(RowType rowType, HdfsConf hdfsConf) { + super(rowType, hdfsConf); + this.hdfsConf = hdfsConf; + List fieldConfList = hdfsConf.getColumn(); + columnNameList = + fieldConfList.stream().map(FieldConf::getName).collect(Collectors.toList()); + for (int i = 0; i < rowType.getFieldCount(); i++) { + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(fieldTypes[i]), fieldTypes[i])); + } + } + + @Override + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, LogicalType type) { + return (rowData, index, group) -> { + if (rowData == null || rowData.isNullAt(index)) { + // do nothing + } else { + serializationConverter.serialize(rowData, index, group); + } + }; + } + + @Override + @SuppressWarnings("unchecked") + public RowData toInternal(RowData input) throws Exception { + ColumnRowData row = new ColumnRowData(input.getArity()); + if (input instanceof GenericRowData) { + GenericRowData genericRowData = (GenericRowData) input; + for (int i = 0; i < input.getArity(); i++) { + row.addField( + (AbstractBaseColumn) + toInternalConverters + .get(i) + .deserialize(genericRowData.getField(i))); + } + } else { + throw new ChunJunRuntimeException( + "Error RowData type, RowData:[" + + input + + "] should be instance of GenericRowData."); + } + return row; + } + + @Override + @SuppressWarnings("unchecked") + public Group toExternal(RowData rowData, Group group) throws Exception { + for (int index = 0; index < hdfsConf.getFullColumnName().size(); index++) { + int columnIndex = hdfsConf.getFullColumnIndexes()[index]; + toExternalConverters.get(columnIndex).serialize(rowData, columnIndex, group); + } + return group; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); + } + + @Override + @SuppressWarnings("all") + protected IDeserializationConverter createInternalConverter(LogicalType logicalType) { + switch (logicalType.getTypeRoot()) { + case NULL: + return val -> null; + case BOOLEAN: + return (IDeserializationConverter) val -> val; + case TINYINT: + return (IDeserializationConverter) val -> val.byteValue(); + case SMALLINT: + return (IDeserializationConverter) val -> val.shortValue(); + case INTEGER: + return (IDeserializationConverter) val -> val; + case BIGINT: + return (IDeserializationConverter) val -> val; + case DATE: + return (IDeserializationConverter) + val -> + (int) + DateUtil.getTimestampFromStr(val) + .toLocalDateTime() + .toLocalDate() + .toEpochDay(); + case FLOAT: + return (IDeserializationConverter) val -> val; + case DOUBLE: + return (IDeserializationConverter) val -> val; + case CHAR: + case VARCHAR: + return (IDeserializationConverter) StringData::fromString; + case DECIMAL: + return (IDeserializationConverter) + val -> DecimalData.fromBigDecimal(val, val.precision(), val.scale()); + case BINARY: + case VARBINARY: + return (IDeserializationConverter) val -> val; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (IDeserializationConverter) + TimestampData::fromTimestamp; + case INTERVAL_DAY_TIME: + case INTERVAL_YEAR_MONTH: + case ARRAY: + case MAP: + case MULTISET: + case ROW: + case RAW: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + default: + throw new UnsupportedTypeException(logicalType); + } + } + + @Override + protected ISerializationConverter createExternalConverter(LogicalType logicalType) { + switch (logicalType.getTypeRoot()) { + case NULL: + return (rowData, index, data) -> {}; + case BOOLEAN: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getBoolean(index)); + case TINYINT: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getByte(index)); + case SMALLINT: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getShort(index)); + case INTEGER: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getInt(index)); + case BIGINT: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getLong(index)); + case DATE: + return (rowData, index, data) -> { + Date date = Date.valueOf(LocalDate.ofEpochDay(rowData.getInt(index))); + data.add(columnNameList.get(index), DateWritable.dateToDays(date)); + }; + case FLOAT: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getFloat(index)); + case DOUBLE: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getDouble(index)); + case CHAR: + case VARCHAR: + return (rowData, index, data) -> + data.add(columnNameList.get(index), rowData.getString(index).toString()); + case DECIMAL: + return (rowData, index, data) -> { + DecimalType decimalType = (DecimalType) logicalType; + int precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + HiveDecimal hiveDecimal = + HiveDecimal.create( + rowData.getDecimal(index, precision, scale).toBigDecimal()); + hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, precision, scale); + if (hiveDecimal == null) { + String msg = + String.format( + "The [%s] data data [%s] precision and scale do not match the metadata:decimal(%s, %s)", + index, precision, scale, rowData); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + data.add( + columnNameList.get(index), + Hive3Util.decimalToBinary(hiveDecimal, precision, scale)); + }; + case BINARY: + case VARBINARY: + return (rowData, index, data) -> + data.add( + columnNameList.get(index), + Binary.fromReusedByteArray(rowData.getBinary(index))); + + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, data) -> { + Timestamp timestamp = ((ColumnRowData) rowData).getField(index).asTimestamp(); + data.add(columnNameList.get(index), Hive3Util.timeToBinary(timestamp)); + }; + case INTERVAL_DAY_TIME: + case INTERVAL_YEAR_MONTH: + case ARRAY: + case MAP: + case MULTISET: + case ROW: + case RAW: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + default: + throw new UnsupportedTypeException(logicalType); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsRawTypeConverter.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsRawTypeConverter.java new file mode 100644 index 0000000000..463ccc69ca --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsRawTypeConverter.java @@ -0,0 +1,91 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.converter; + +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.BinaryType; + +import java.util.Locale; + +/** @author liuliu 2022/3/22 */ +public class HdfsRawTypeConverter { + public static DataType apply(String type) throws UnsupportedTypeException { + type = type.toUpperCase(Locale.ENGLISH); + int left = type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL); + int right = type.indexOf(ConstantValue.RIGHT_PARENTHESIS_SYMBOL); + String leftStr = type; + String rightStr = null; + if (left > 0 && right > 0) { + leftStr = type.substring(0, left); + rightStr = type.substring(left + 1, type.length() - 1); + } + switch (leftStr) { + case "BOOLEAN": + return DataTypes.BOOLEAN(); + case "TINYINT": + return DataTypes.TINYINT(); + case "SMALLINT": + return DataTypes.SMALLINT(); + case "INT": + return DataTypes.INT(); + case "BIGINT": + return DataTypes.BIGINT(); + case "FLOAT": + return DataTypes.FLOAT(); + case "DOUBLE": + return DataTypes.DOUBLE(); + case "DECIMAL": + if (rightStr != null) { + String[] split = rightStr.split(ConstantValue.COMMA_SYMBOL); + if (split.length == 2) { + return DataTypes.DECIMAL( + Integer.parseInt(split[0].trim()), + Integer.parseInt(split[1].trim())); + } + } + return DataTypes.DECIMAL(38, 18); + case "STRING": + case "VARCHAR": + case "CHAR": + return DataTypes.STRING(); + case "BINARY": + return DataTypes.BINARY(BinaryType.DEFAULT_LENGTH); + case "TIMESTAMP": + if (rightStr != null) { + String[] split = rightStr.split(ConstantValue.COMMA_SYMBOL); + if (split.length == 1) { + return DataTypes.TIMESTAMP(Integer.parseInt(split[0].trim())); + } + } + return DataTypes.TIMESTAMP(6); + case "DATE": + return DataTypes.DATE(); + case "ARRAY": + case "MAP": + case "STRUCT": + case "UNION": + default: + throw new UnsupportedTypeException(type); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsTextColumnConverter.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsTextColumnConverter.java new file mode 100644 index 0000000000..60edb5168e --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsTextColumnConverter.java @@ -0,0 +1,223 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.converter; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.IDeserializationConverter; +import com.dtstack.chunjun.converter.ISerializationConverter; +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.ColumnRowData; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.ByteColumn; +import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.SqlDateColumn; +import com.dtstack.chunjun.element.column.StringColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; +import com.dtstack.chunjun.util.DateUtil; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.hadoop.hive.common.type.HiveDecimal; + +import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; + +/** @author liuliu 2022/3/22 */ +public class HdfsTextColumnConverter + extends AbstractRowConverter, LogicalType> { + + HdfsConf hdfsConf; + + public HdfsTextColumnConverter(RowType rowType, HdfsConf hdfsConf) { + super(rowType, hdfsConf); + this.hdfsConf = hdfsConf; + for (int i = 0; i < rowType.getFieldCount(); i++) { + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(fieldTypes[i]), fieldTypes[i])); + } + } + + @Override + public RowData toInternal(RowData input) throws Exception { + ColumnRowData row = new ColumnRowData(input.getArity()); + if (input instanceof GenericRowData) { + List fieldConfList = commonConf.getColumn(); + GenericRowData genericRowData = (GenericRowData) input; + for (int i = 0; i < fieldConfList.size(); i++) { + row.addField( + assembleFieldProps( + fieldConfList.get(i), + (AbstractBaseColumn) + toInternalConverters + .get(i) + .deserialize(genericRowData.getField(i)))); + } + } else { + throw new ChunJunRuntimeException( + "Error RowData type, RowData:[" + + input + + "] should be instance of GenericRowData."); + } + return row; + } + + @Override + public List toExternal(RowData rowData, List output) throws Exception { + for (int index = 0; index < hdfsConf.getFullColumnName().size(); index++) { + int columnIndex = hdfsConf.getFullColumnIndexes()[index]; + if (columnIndex == -1) { + output.add(""); + } else { + toExternalConverters.get(columnIndex).serialize(rowData, columnIndex, output); + } + } + return output; + } + + @Override + protected ISerializationConverter> wrapIntoNullableExternalConverter( + ISerializationConverter> serializationConverter, LogicalType type) { + return (rowData, index, data) -> { + if (rowData == null || rowData.isNullAt(index)) { + data.add(null); + } else { + serializationConverter.serialize(rowData, index, data); + } + }; + } + + @Override + protected IDeserializationConverter createInternalConverter( + LogicalType type) { + switch (type.getTypeRoot()) { + case BOOLEAN: + return val -> new BooleanColumn(Boolean.parseBoolean(val)); + case TINYINT: + return val -> new ByteColumn(Byte.parseByte(val)); + case SMALLINT: + case INTEGER: + case BIGINT: + case FLOAT: + case DOUBLE: + case DECIMAL: + return BigDecimalColumn::new; + case VARCHAR: + return StringColumn::new; + case BINARY: + return val -> new BytesColumn(val.getBytes(StandardCharsets.UTF_8)); + case DATE: + return val -> { + Timestamp timestamp = DateUtil.getTimestampFromStr(val); + if (timestamp == null) { + return new SqlDateColumn(null); + } else { + return new SqlDateColumn( + Date.valueOf(timestamp.toLocalDateTime().toLocalDate())); + } + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return val -> { + try { + return new TimestampColumn( + Timestamp.valueOf(val), DateUtil.getPrecisionFromTimestampStr(val)); + } catch (Exception e) { + return new TimestampColumn(DateUtil.getTimestampFromStr(val), 0); + } + }; + default: + throw new UnsupportedTypeException(type); + } + } + + @Override + protected ISerializationConverter> createExternalConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case BOOLEAN: + return (rowData, index, data) -> + data.add( + String.valueOf( + ((ColumnRowData) rowData).getField(index).asBoolean())); + case TINYINT: + return (rowData, index, data) -> data.add(String.valueOf(rowData.getByte(index))); + case SMALLINT: + case INTEGER: + return (rowData, index, data) -> + data.add( + String.valueOf( + ((ColumnRowData) rowData).getField(index).asYearInt())); + case BIGINT: + return (rowData, index, data) -> + data.add( + String.valueOf(((ColumnRowData) rowData).getField(index).asLong())); + case FLOAT: + return (rowData, index, data) -> + data.add( + String.valueOf( + ((ColumnRowData) rowData).getField(index).asFloat())); + case DOUBLE: + return (rowData, index, data) -> + data.add( + String.valueOf( + ((ColumnRowData) rowData).getField(index).asDouble())); + case DECIMAL: + return (rowData, index, data) -> + data.add( + String.valueOf( + HiveDecimal.create( + ((ColumnRowData) rowData) + .getField(index) + .asBigDecimal()))); + case VARCHAR: + return (rowData, index, data) -> + data.add( + String.valueOf( + ((ColumnRowData) rowData).getField(index).asString())); + case DATE: + return (rowData, index, data) -> + data.add( + String.valueOf( + ((ColumnRowData) rowData).getField(index).asSqlDate())); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, data) -> + data.add(((ColumnRowData) rowData).getField(index).asTimestampStr()); + case BINARY: + return (rowData, index, data) -> + data.add( + Arrays.toString( + ((ColumnRowData) rowData).getField(index).asBytes())); + default: + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsTextRowConverter.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsTextRowConverter.java new file mode 100644 index 0000000000..db7fbfd273 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/converter/HdfsTextRowConverter.java @@ -0,0 +1,241 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.converter; + +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.IDeserializationConverter; +import com.dtstack.chunjun.converter.ISerializationConverter; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; +import com.dtstack.chunjun.util.DateUtil; + +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.format.DateTimeFormatter; +import java.time.temporal.TemporalQueries; +import java.util.Arrays; +import java.util.List; + +/** @author liuliu 2022/3/22 */ +public class HdfsTextRowConverter + extends AbstractRowConverter, LogicalType> { + + HdfsConf hdfsConf; + + public HdfsTextRowConverter(RowType rowType, HdfsConf hdfsConf) { + super(rowType, hdfsConf); + this.hdfsConf = hdfsConf; + for (int i = 0; i < rowType.getFieldCount(); i++) { + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(fieldTypes[i]), fieldTypes[i])); + } + } + + @Override + @SuppressWarnings("unchecked") + public RowData toInternal(RowData input) throws Exception { + GenericRowData row = new GenericRowData(input.getArity()); + if (input instanceof GenericRowData) { + GenericRowData genericRowData = (GenericRowData) input; + for (int i = 0; i < input.getArity(); i++) { + row.setField( + i, toInternalConverters.get(i).deserialize(genericRowData.getField(i))); + } + } else { + throw new ChunJunRuntimeException( + "Error RowData type, RowData:[" + + input + + "] should be instance of GenericRowData."); + } + return row; + } + + @Override + @SuppressWarnings("unchecked") + public List toExternal(RowData rowData, List data) throws Exception { + for (int index = 0; index < hdfsConf.getFullColumnName().size(); index++) { + int columnIndex = hdfsConf.getFullColumnIndexes()[index]; + toExternalConverters.get(columnIndex).serialize(rowData, columnIndex, data); + } + return data; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); + } + + @Override + @SuppressWarnings("unchecked") + public ISerializationConverter> wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, LogicalType type) { + return (rowData, index, data) -> { + if (rowData == null + || rowData.isNullAt(index) + || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) { + data.add(null); + } else { + serializationConverter.serialize(rowData, index, data); + } + }; + } + + @Override + protected IDeserializationConverter createInternalConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return val -> null; + case BOOLEAN: + return (IDeserializationConverter) Boolean::getBoolean; + case TINYINT: + return (IDeserializationConverter) Byte::parseByte; + case SMALLINT: + return (IDeserializationConverter) Short::parseShort; + case INTEGER: + return (IDeserializationConverter) Integer::parseInt; + case BIGINT: + return (IDeserializationConverter) Long::parseLong; + case DATE: + return (IDeserializationConverter) + val -> { + LocalDate date = + DateTimeFormatter.ISO_LOCAL_DATE + .parse(val) + .query(TemporalQueries.localDate()); + return (int) date.toEpochDay(); + }; + case FLOAT: + return (IDeserializationConverter) Float::parseFloat; + case DOUBLE: + return (IDeserializationConverter) Double::parseDouble; + case CHAR: + case VARCHAR: + return (IDeserializationConverter) StringData::fromString; + case DECIMAL: + return (IDeserializationConverter) + val -> { + final int precision = ((DecimalType) type).getPrecision(); + final int scale = ((DecimalType) type).getScale(); + return DecimalData.fromBigDecimal( + new BigDecimal(val), precision, scale); + }; + case BINARY: + case VARBINARY: + return (IDeserializationConverter) + val -> val.getBytes(StandardCharsets.UTF_8); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (IDeserializationConverter) + val -> + TimestampData.fromTimestamp( + new Timestamp(DateUtil.stringToDate(val).getTime())); + case INTERVAL_DAY_TIME: + case INTERVAL_YEAR_MONTH: + case ARRAY: + case MAP: + case MULTISET: + case ROW: + case RAW: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + default: + throw new UnsupportedTypeException(type); + } + } + + @Override + protected ISerializationConverter> createExternalConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return (rowData, index, data) -> data.add(null); + case BOOLEAN: + return (rowData, index, data) -> + data.add(String.valueOf(rowData.getBoolean(index))); + case TINYINT: + return (rowData, index, data) -> data.add(String.valueOf(rowData.getByte(index))); + case SMALLINT: + return (rowData, index, data) -> data.add(String.valueOf(rowData.getShort(index))); + case INTEGER: + return (rowData, index, data) -> data.add(String.valueOf(rowData.getInt(index))); + case BIGINT: + return (rowData, index, data) -> data.add(String.valueOf(rowData.getLong(index))); + case DATE: + return (rowData, index, data) -> + data.add( + String.valueOf( + Date.valueOf(LocalDate.ofEpochDay(rowData.getInt(index))))); + case FLOAT: + return (rowData, index, data) -> data.add(String.valueOf(rowData.getFloat(index))); + case DOUBLE: + return (rowData, index, data) -> data.add(String.valueOf(rowData.getDouble(index))); + case CHAR: + case VARCHAR: + return (rowData, index, data) -> data.add(String.valueOf(rowData.getString(index))); + case DECIMAL: + return (rowData, index, data) -> + data.add( + String.valueOf( + rowData.getDecimal( + index, + ((DecimalType) type).getPrecision(), + ((DecimalType) type).getScale()))); + case BINARY: + case VARBINARY: + return (rowData, index, data) -> + data.add(Arrays.toString(rowData.getBinary(index))); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, data) -> + data.add( + String.valueOf( + rowData.getTimestamp( + index, + ((TimestampType) type).getPrecision()) + .toTimestamp())); + case INTERVAL_DAY_TIME: + case INTERVAL_YEAR_MONTH: + case ARRAY: + case MAP: + case MULTISET: + case ROW: + case RAW: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + default: + throw new UnsupportedTypeException(type); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/enums/CompressType.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/enums/CompressType.java new file mode 100644 index 0000000000..215a73b6d5 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/enums/CompressType.java @@ -0,0 +1,95 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.enums; + +import org.apache.commons.lang.StringUtils; + +/** + * Date: 2021/06/09 Company: www.dtstack.com + * + * @author tudou + */ +public enum CompressType { + /** text file */ + TEXT_GZIP("GZIP", "text", ".gz", 0.331F), + TEXT_BZIP2("BZIP2", "text", ".bz2", 0.259F), + TEXT_NONE("NONE", "text", "", 0.637F), + + /** orc file */ + ORC_SNAPPY("SNAPPY", "orc", ".snappy", 0.233F), + ORC_GZIP("GZIP", "orc", ".gz", 1.0F), + ORC_BZIP("BZIP", "orc", ".bz", 1.0F), + ORC_LZ4("LZ4", "orc", ".lz4", 1.0F), + ORC_NONE("NONE", "orc", "", 0.233F), + + /** parquet file */ + PARQUET_SNAPPY("SNAPPY", "parquet", ".snappy", 0.274F), + PARQUET_GZIP("GZIP", "parquet", ".gz", 1.0F), + PARQUET_LZO("LZO", "parquet", ".lzo", 1.0F), + PARQUET_NONE("NONE", "parquet", "", 1.0F); + + private final String type; + + private final String fileType; + + private final String suffix; + + private final float deviation; + + CompressType(String type, String fileType, String suffix, float deviation) { + this.type = type; + this.fileType = fileType; + this.suffix = suffix; + this.deviation = deviation; + } + + public static CompressType getByTypeAndFileType(String type, String fileType) { + if (StringUtils.isEmpty(type)) { + if ("PARQUET".equalsIgnoreCase(fileType)) { + type = "SNAPPY"; + } else { + type = "NONE"; + } + } + + for (CompressType value : CompressType.values()) { + if (value.getType().equalsIgnoreCase(type) + && value.getFileType().equalsIgnoreCase(fileType)) { + return value; + } + } + + throw new IllegalArgumentException("No enum constant " + type); + } + + public String getType() { + return type; + } + + public String getFileType() { + return fileType; + } + + public String getSuffix() { + return suffix; + } + + public float getDeviation() { + return deviation; + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/enums/FileType.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/enums/FileType.java new file mode 100644 index 0000000000..03c22bb140 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/enums/FileType.java @@ -0,0 +1,51 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.enums; + +import com.dtstack.chunjun.throwable.UnsupportedTypeException; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Locale; + +/** + * Date: 2021/06/17 Company: www.dtstack.com + * + * @author tudou + */ +public enum FileType { + TEXT, + ORC, + PARQUET; + + public static FileType getByName(String fileType) { + if (StringUtils.isBlank(fileType)) { + throw new IllegalArgumentException("fileType cannot be null or empty."); + } + switch (fileType.toLowerCase(Locale.ENGLISH)) { + case "text": + return TEXT; + case "orc": + return ORC; + case "parquet": + return PARQUET; + default: + throw new UnsupportedTypeException(fileType); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsOrcInputSplit.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsOrcInputSplit.java new file mode 100644 index 0000000000..ae234cb3d1 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsOrcInputSplit.java @@ -0,0 +1,65 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.inputSplit; + +import org.apache.flink.core.io.InputSplit; + +import org.apache.hadoop.hive.ql.io.orc.OrcSplit; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; + +/** @author liuliu 2022/3/23 */ +public class HdfsOrcInputSplit implements InputSplit { + private static final long serialVersionUID = 1L; + int splitNumber; + byte[] orcSplitData; + + public HdfsOrcInputSplit(OrcSplit orcSplit, int splitNumber) throws IOException { + this.splitNumber = splitNumber; + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + orcSplit.write(dos); + orcSplitData = baos.toByteArray(); + baos.close(); + dos.close(); + } + + public OrcSplit getOrcSplit() throws IOException { + ByteArrayInputStream bais = new ByteArrayInputStream(orcSplitData); + DataInputStream dis = new DataInputStream(bais); + // TODO 修改 分片代码 + OrcSplit orcSplit = + new OrcSplit( + null, null, 0, 0, null, null, false, false, new ArrayList(), 0, 0, null); + orcSplit.readFields(dis); + bais.close(); + dis.close(); + return orcSplit; + } + + @Override + public int getSplitNumber() { + return splitNumber; + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsParquetInputSplit.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsParquetInputSplit.java new file mode 100644 index 0000000000..b1c8409d02 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsParquetInputSplit.java @@ -0,0 +1,44 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.inputSplit; + +import org.apache.flink.core.io.InputSplit; + +import java.util.List; + +/** @author liuliu 2022/3/23 */ +public class HdfsParquetInputSplit implements InputSplit { + private final int splitNumber; + + private final List paths; + + public HdfsParquetInputSplit(int splitNumber, List paths) { + this.splitNumber = splitNumber; + this.paths = paths; + } + + @Override + public int getSplitNumber() { + return splitNumber; + } + + public List getPaths() { + return paths; + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsTextInputSplit.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsTextInputSplit.java new file mode 100644 index 0000000000..85aaf30d35 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/inputSplit/HdfsTextInputSplit.java @@ -0,0 +1,62 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.inputSplit; + +import org.apache.flink.core.io.InputSplit; + +import org.apache.commons.io.output.ByteArrayOutputStream; +import org.apache.hadoop.mapred.FileSplit; + +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** @author liuliu 2022/3/23 */ +public class HdfsTextInputSplit implements InputSplit { + private static final long serialVersionUID = 1L; + int splitNumber; + byte[] textSplitData; + + public HdfsTextInputSplit(org.apache.hadoop.mapred.InputSplit split, int splitNumber) + throws IOException { + this.splitNumber = splitNumber; + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + split.write(dos); + textSplitData = baos.toByteArray(); + baos.close(); + dos.close(); + } + + public org.apache.hadoop.mapred.InputSplit getTextSplit() throws IOException { + ByteArrayInputStream bais = new ByteArrayInputStream(textSplitData); + DataInputStream dis = new DataInputStream(bais); + org.apache.hadoop.mapred.InputSplit split = new FileSplit(null, 0L, 0L, (String[]) null); + split.readFields(dis); + bais.close(); + dis.close(); + return split; + } + + @Override + public int getSplitNumber() { + return splitNumber; + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/BaseHdfsOutputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/BaseHdfsOutputFormat.java new file mode 100644 index 0000000000..bd7166edda --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/BaseHdfsOutputFormat.java @@ -0,0 +1,276 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.sink; + +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.enums.CompressType; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.sink.format.BaseFileOutputFormat; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.util.ColumnTypeUtil; +import com.dtstack.chunjun.util.PluginUtil; + +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.common.functions.RuntimeContext; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** @author liuliu 2022/3/23 */ +public abstract class BaseHdfsOutputFormat extends BaseFileOutputFormat { + protected FileSystem fs; + protected HdfsConf hdfsConf; + + protected List fullColumnNameList; + protected List fullColumnTypeList; + protected int[] fullColumnIndexes; + protected Configuration conf; + protected transient Map decimalColInfo; + protected CompressType compressType; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + // 这里休眠一段时间是为了避免reader和writer或者多个任务在同一个taskManager里同时认证kerberos + if (Hive3Util.isOpenKerberos(hdfsConf.getHadoopConfig())) { + try { + Thread.sleep(5000L + (long) (10000 * Math.random())); + } catch (Exception e) { + LOG.warn("", e); + } + } + super.openInternal(taskNumber, numTasks); + } + + @Override + protected void initVariableFields() { + compressType = getCompressType(); + fullColumnNameList = hdfsConf.getFullColumnName(); + fullColumnTypeList = hdfsConf.getFullColumnType(); + fullColumnIndexes = hdfsConf.getFullColumnIndexes(); + super.initVariableFields(); + } + + @Override + protected void checkOutputDir() { + try { + Path dir = new Path(tmpPath); + if (fs == null) { + openSource(); + } + if (fs.exists(dir)) { + if (fs.isFile(dir)) { + throw new ChunJunRuntimeException(String.format("dir:[%s] is a file", tmpPath)); + } + } else { + fs.mkdirs(dir); + } + } catch (IOException e) { + throw new ChunJunRuntimeException( + "cannot check or create temp directory: " + tmpPath, e); + } + } + + @Override + protected void deleteDataDir() { + deleteDirectory(outputFilePath); + } + + @Override + protected void deleteTmpDataDir() { + deleteDirectory(tmpPath); + } + + @Override + protected void openSource() { + conf = Hive3Util.getConfiguration(hdfsConf.getHadoopConfig(), hdfsConf.getDefaultFS()); + RuntimeContext runtimeContext = null; + try { + runtimeContext = getRuntimeContext(); + } catch (IllegalStateException e) { + // ignore + } + DistributedCache distributedCache; + if (runtimeContext == null) { + distributedCache = PluginUtil.createDistributedCacheFromContextClassLoader(); + } else { + distributedCache = runtimeContext.getDistributedCache(); + } + try { + fs = + Hive3Util.getFileSystem( + hdfsConf.getHadoopConfig(), hdfsConf.getDefaultFS(), distributedCache); + } catch (Exception e) { + throw new ChunJunRuntimeException("can't init fileSystem", e); + } + } + + @Override + public String getExtension() { + return compressType.getSuffix(); + } + + @Override + protected long getCurrentFileSize() { + String path = tmpPath + File.separatorChar + currentFileName; + try { + if (hdfsConf.getMaxFileSize() > ConstantValue.STORE_SIZE_G) { + return fs.getFileStatus(new Path(path)).getLen(); + } else { + return fs.open(new Path(path)).available(); + } + } catch (IOException e) { + throw new ChunJunRuntimeException("can't get file size from hdfs, file = " + path, e); + } + } + + @Override + protected List copyTmpDataFileToDir() { + String filePrefix = jobId + "_" + taskNumber; + PathFilter pathFilter = path -> path.getName().startsWith(filePrefix); + Path dir = new Path(outputFilePath); + Path tmpDir = new Path(tmpPath); + String currentFilePath = ""; + List copyList = new ArrayList<>(); + try { + FileStatus[] dataFiles = fs.listStatus(tmpDir, pathFilter); + for (FileStatus dataFile : dataFiles) { + currentFilePath = dataFile.getPath().getName(); + FileUtil.copy(fs, dataFile.getPath(), fs, dir, false, conf); + copyList.add(currentFilePath); + LOG.info("copy temp file:{} to dir:{}", currentFilePath, dir); + } + } catch (Exception e) { + throw new ChunJunRuntimeException( + String.format( + "can't copy temp file:[%s] to dir:[%s]", + currentFilePath, outputFilePath), + e); + } + return copyList; + } + + @Override + protected void deleteDataFiles(List preCommitFilePathList, String path) { + String currentFilePath = ""; + try { + for (String fileName : this.preCommitFilePathList) { + currentFilePath = path + File.separatorChar + fileName; + Path commitFilePath = new Path(currentFilePath); + fs.delete(commitFilePath, true); + LOG.info("delete file:{}", currentFilePath); + } + } catch (IOException e) { + throw new ChunJunRuntimeException( + String.format("can't delete commit file:[%s]", currentFilePath), e); + } + } + + @Override + protected void moveAllTmpDataFileToDir() { + if (fs == null) { + openSource(); + } + String currentFilePath = ""; + try { + Path dir = new Path(outputFilePath); + Path tmpDir = new Path(tmpPath); + + FileStatus[] dataFiles = fs.listStatus(tmpDir); + for (FileStatus dataFile : dataFiles) { + currentFilePath = dataFile.getPath().getName(); + fs.rename(dataFile.getPath(), dir); + LOG.info("move temp file:{} to dir:{}", dataFile.getPath(), dir); + } + fs.delete(tmpDir, true); + } catch (IOException e) { + throw new ChunJunRuntimeException( + String.format( + "can't move file:[%s] to dir:[%s]", currentFilePath, outputFilePath), + e); + } + } + + @Override + protected void closeSource() { + try { + if (fs != null) { + fs.close(); + fs = null; + } + } catch (IOException e) { + throw new ChunJunRuntimeException("can't close source.", e); + } + } + + @Override + public float getDeviation() { + return compressType.getDeviation(); + } + + /** get file compress type */ + protected abstract CompressType getCompressType(); + + protected void deleteDirectory(String path) { + deleteDirectory(path, true); + } + + protected void deleteDirectory(String path, boolean containDir) { + LOG.info("start to delete directory:{}", path); + try { + Path dir = new Path(path); + if (fs == null) { + openSource(); + } + if (fs.exists(dir)) { + if (fs.isFile(dir)) { + throw new ChunJunRuntimeException(String.format("dir:[%s] is a file", path)); + } + if (containDir) { + fs.delete(dir, true); + } else if (fs.isDirectory(dir)) { + FileStatus[] fileStatuses = fs.listStatus(dir); + for (FileStatus status : fileStatuses) { + fs.delete(status.getPath(), true); + } + } + } + } catch (IOException e) { + throw new ChunJunRuntimeException("cannot delete directory: " + path, e); + } + } + + public HdfsConf getHdfsConf() { + return hdfsConf; + } + + public void setHdfsConf(HdfsConf hdfsConf) { + this.hdfsConf = hdfsConf; + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsOrcOutputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsOrcOutputFormat.java new file mode 100644 index 0000000000..9a688b25a8 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsOrcOutputFormat.java @@ -0,0 +1,239 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.sink; + +import com.dtstack.chunjun.connector.hive3.enums.CompressType; +import com.dtstack.chunjun.connector.hive3.enums.FileType; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.enums.ColumnType; +import com.dtstack.chunjun.enums.SizeUnitType; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.WriteRecordException; +import com.dtstack.chunjun.util.ColumnTypeUtil; +import com.dtstack.chunjun.util.ExceptionUtil; +import com.dtstack.chunjun.util.ReflectionUtils; + +import org.apache.flink.table.data.RowData; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.orc.OrcFile; +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.compress.BZip2Codec; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.io.compress.Lz4Codec; +import org.apache.hadoop.io.compress.SnappyCodec; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +/** @author liuliu 2022/3/27 */ +public class HdfsOrcOutputFormat extends BaseHdfsOutputFormat { + private RecordWriter recordWriter; + private OrcSerde orcSerde; + private StructObjectInspector inspector; + private FileOutputFormat outputFormat; + private JobConf jobConf; + private static final ColumnTypeUtil.DecimalInfo ORC_DEFAULT_DECIMAL_INFO = + new ColumnTypeUtil.DecimalInfo( + HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE); + + // kerberos 认证 + protected transient UserGroupInformation ugi; + protected boolean openKerberos; + public static final String HADOOP_USER_NAME = "HADOOP_USER_NAME"; + public static final String KEY_PRINCIPAL = "principal"; + + protected final Logger LOG = LoggerFactory.getLogger(HdfsOrcOutputFormat.class); + + @Override + protected void openSource() { + super.openSource(); + orcSerde = new OrcSerde(); + outputFormat = new org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat(); + jobConf = new JobConf(conf); + FileOutputFormat.setOutputCompressorClass(jobConf, getOrcCodecClass(compressType)); + + List fullColTypeList = new ArrayList<>(); + + decimalColInfo = new HashMap<>((fullColumnTypeList.size() << 2) / 3); + for (int i = 0; i < fullColumnTypeList.size(); i++) { + String columnType = fullColumnTypeList.get(i); + + ColumnTypeUtil.DecimalInfo decimalInfo = null; + if (ColumnTypeUtil.isDecimalType(columnType)) { + decimalInfo = ColumnTypeUtil.getDecimalInfo(columnType, ORC_DEFAULT_DECIMAL_INFO); + decimalColInfo.put(fullColumnNameList.get(i), decimalInfo); + } + + ColumnType type = ColumnType.getType(columnType); + fullColTypeList.add(Hive3Util.columnTypeToObjectInspector(type, decimalInfo)); + } + + this.inspector = + ObjectInspectorFactory.getStandardStructObjectInspector( + fullColumnNameList, fullColTypeList); + } + + @Override + protected CompressType getCompressType() { + return CompressType.getByTypeAndFileType(hdfsConf.getCompress(), FileType.ORC.name()); + } + + protected Class getOrcCodecClass(CompressType compressType) { + if (CompressType.ORC_SNAPPY.equals(compressType)) { + return SnappyCodec.class; + } else if (CompressType.ORC_BZIP.equals(compressType)) { + return BZip2Codec.class; + } else if (CompressType.ORC_GZIP.equals(compressType)) { + return GzipCodec.class; + } else if (CompressType.ORC_LZ4.equals(compressType)) { + return Lz4Codec.class; + } else { + return DefaultCodec.class; + } + } + + @Override + protected void nextBlock() { + super.nextBlock(); + if (recordWriter != null) { + return; + } + try { + String currentBlockTmpPath = tmpPath + File.separatorChar + currentFileName; + recordWriter = + outputFormat.getRecordWriter(null, jobConf, currentBlockTmpPath, Reporter.NULL); + currentFileIndex++; + + setFs(); + LOG.info("nextBlock:Current block writer record:" + rowsOfCurrentBlock); + LOG.info("Current block file name:" + currentBlockTmpPath); + } catch (Exception e) { + throw new RuntimeException( + Hive3Util.parseErrorMsg(null, ExceptionUtil.getErrorMessage(e)), e); + } + } + + /** + * 数据源开启kerberos时 如果这里不通过反射对 writerOptions 赋值fs,则在recordWriter.writer时 会初始化一个fs 此fs不在ugi里获取的 + * 导致开启了kerberos的数据源在checkpoint时进行 recordWriter.close() 操作,会出现kerberos认证错误 + */ + private void setFs() throws IllegalAccessException { + if (Hive3Util.isOpenKerberos(hdfsConf.getHadoopConfig())) { + Field declaredField = ReflectionUtils.getDeclaredField(recordWriter, "options"); + assert declaredField != null; + declaredField.setAccessible(true); + OrcFile.WriterOptions writerOptions = + (OrcFile.WriterOptions) declaredField.get(recordWriter); + writerOptions.fileSystem(fs); + declaredField.setAccessible(false); + } + } + + @Override + public void writeSingleRecordToFile(RowData row) throws WriteRecordException { + if (recordWriter == null) { + nextBlock(); + } + + List recordList = new ArrayList<>(); + int i = 0; + try { + rowConverter.toExternal(row, recordList); + } catch (Exception e) { + if (e instanceof WriteRecordException) { + throw (WriteRecordException) e; + } else { + throw new WriteRecordException(recordConvertDetailErrorMessage(i, row), e, i, row); + } + } + + try { + this.recordWriter.write( + NullWritable.get(), this.orcSerde.serialize(recordList, this.inspector)); + rowsOfCurrentBlock++; + + lastRow = row; + } catch (IOException e) { + String errorMessage = + Hive3Util.parseErrorMsg( + String.format("writer hdfs error,row:{%s}", row), + ExceptionUtil.getErrorMessage(e)); + LOG.error(errorMessage); + throw new WriteRecordException(errorMessage, e); + } + } + + @Override + // todo the deviation needs to be calculated accurately + protected long getCurrentFileSize() { + return (long) (bytesWriteCounter.getLocalValue() * getDeviation()); + } + + @Override + protected void flushDataInternal() { + LOG.info( + "Close current orc record writer, write data size:[{}]", + SizeUnitType.readableFileSize(bytesWriteCounter.getLocalValue())); + + try { + if (recordWriter != null) { + recordWriter.close(Reporter.NULL); + recordWriter = null; + } + } catch (IOException e) { + throw new ChunJunRuntimeException( + Hive3Util.parseErrorMsg( + "error to flush stream.", ExceptionUtil.getErrorMessage(e)), + e); + } + } + + @Override + protected void closeSource() { + try { + LOG.info("close:Current block writer record:" + rowsOfCurrentBlock); + RecordWriter rw = this.recordWriter; + if (rw != null) { + rw.close(Reporter.NULL); + this.recordWriter = null; + } + } catch (IOException e) { + throw new ChunJunRuntimeException("close stream error.", e); + } finally { + super.closeSource(); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsParquetOutputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsParquetOutputFormat.java new file mode 100644 index 0000000000..95cd13416f --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsParquetOutputFormat.java @@ -0,0 +1,276 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.sink; + +import com.dtstack.chunjun.connector.hive3.enums.CompressType; +import com.dtstack.chunjun.connector.hive3.enums.FileType; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.enums.SizeUnitType; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.WriteRecordException; +import com.dtstack.chunjun.util.ColumnTypeUtil; +import com.dtstack.chunjun.util.ExceptionUtil; + +import org.apache.flink.table.data.RowData; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.SimpleGroupFactory; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.example.ExampleParquetWriter; +import org.apache.parquet.hadoop.example.GroupWriteSupport; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Types; + +import java.io.File; +import java.io.IOException; +import java.security.PrivilegedAction; +import java.util.HashMap; +import java.util.Locale; + +/** @author liuliu 2022/3/23 */ +public class HdfsParquetOutputFormat extends BaseHdfsOutputFormat { + + private static final ColumnTypeUtil.DecimalInfo PARQUET_DEFAULT_DECIMAL_INFO = + new ColumnTypeUtil.DecimalInfo(10, 0); + private SimpleGroupFactory groupFactory; + private ParquetWriter writer; + private MessageType schema; + + @Override + protected void openSource() { + super.openSource(); + + schema = buildSchema(); + GroupWriteSupport.setSchema(schema, conf); + groupFactory = new SimpleGroupFactory(schema); + } + + @Override + protected void nextBlock() { + super.nextBlock(); + + if (writer != null) { + return; + } + + try { + String currentBlockTmpPath = tmpPath + File.separatorChar + currentFileName; + Path writePath = new Path(currentBlockTmpPath); + + // Compatible with old code + CompressionCodecName compressionCodecName; + switch (compressType) { + case PARQUET_SNAPPY: + compressionCodecName = CompressionCodecName.SNAPPY; + break; + case PARQUET_GZIP: + compressionCodecName = CompressionCodecName.GZIP; + break; + case PARQUET_LZO: + compressionCodecName = CompressionCodecName.LZO; + break; + default: + compressionCodecName = CompressionCodecName.UNCOMPRESSED; + } + + ExampleParquetWriter.Builder builder = + ExampleParquetWriter.builder(writePath) + .withWriteMode(ParquetFileWriter.Mode.CREATE) + .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_1_0) + .withCompressionCodec(compressionCodecName) + .withConf(conf) + .withType(schema) + .withDictionaryEncoding(hdfsConf.isEnableDictionary()) + .withRowGroupSize(hdfsConf.getRowGroupSize()); + + // 开启kerberos 需要在ugi里进行build + if (Hive3Util.isOpenKerberos(hdfsConf.getHadoopConfig())) { + UserGroupInformation ugi = + Hive3Util.getUGI( + hdfsConf.getHadoopConfig(), + hdfsConf.getDefaultFS(), + getRuntimeContext().getDistributedCache()); + ugi.doAs( + (PrivilegedAction) + () -> { + try { + writer = builder.build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return null; + }); + } else { + writer = builder.build(); + } + currentFileIndex++; + } catch (Exception e) { + throw new ChunJunRuntimeException( + Hive3Util.parseErrorMsg(null, ExceptionUtil.getErrorMessage(e)), e); + } + } + + @Override + public void flushDataInternal() { + LOG.info( + "Close current parquet record writer, write data size:[{}]", + SizeUnitType.readableFileSize(bytesWriteCounter.getLocalValue())); + try { + if (writer != null) { + writer.close(); + writer = null; + } + } catch (IOException e) { + throw new ChunJunRuntimeException( + Hive3Util.parseErrorMsg( + "error to flush stream.", ExceptionUtil.getErrorMessage(e)), + e); + } + } + + @Override + @SuppressWarnings("unchecked") + public void writeSingleRecordToFile(RowData rowData) throws WriteRecordException { + if (writer == null) { + nextBlock(); + } + Group group = groupFactory.newGroup(); + try { + rowConverter.toExternal(rowData, group); + } catch (Exception e) { + String errorMessage = + Hive3Util.parseErrorMsg( + String.format("writer hdfs error,rowData:{%s}", rowData), + ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(errorMessage, e, -1, rowData); + } + + try { + writer.write(group); + rowsOfCurrentBlock++; + lastRow = rowData; + } catch (IOException e) { + throw new WriteRecordException( + String.format("Data writing to hdfs is abnormal,rowData:{%s}", rowData), e); + } + } + + @Override + protected void closeSource() { + try { + LOG.info("close:Current block writer record:" + rowsOfCurrentBlock); + if (writer != null) { + writer.close(); + } + } catch (IOException e) { + throw new ChunJunRuntimeException("close stream error.", e); + } finally { + super.closeSource(); + } + } + + @Override + public CompressType getCompressType() { + return CompressType.getByTypeAndFileType(hdfsConf.getCompress(), FileType.PARQUET.name()); + } + + @SuppressWarnings("all") + private MessageType buildSchema() { + decimalColInfo = new HashMap<>(16); + Types.MessageTypeBuilder typeBuilder = Types.buildMessage(); + for (int i = 0; i < fullColumnNameList.size(); i++) { + String name = fullColumnNameList.get(i); + String colType = fullColumnTypeList.get(i).toLowerCase(Locale.ENGLISH); + int left = colType.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL); + int right = colType.indexOf(ConstantValue.RIGHT_PARENTHESIS_SYMBOL); + if (left > 0 && right > 0) { + colType = colType.substring(0, left); + } + switch (colType) { + case "tinyint": + case "smallint": + case "int": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.INT32).named(name); + break; + case "bigint": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.INT64).named(name); + break; + case "float": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.FLOAT).named(name); + break; + case "double": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.DOUBLE).named(name); + break; + case "binary": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BINARY).named(name); + break; + case "char": + case "varchar": + case "string": + typeBuilder + .optional(PrimitiveType.PrimitiveTypeName.BINARY) + .as(OriginalType.UTF8) + .named(name); + break; + case "boolean": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BOOLEAN).named(name); + break; + case "timestamp": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.INT96).named(name); + break; + case "date": + typeBuilder + .optional(PrimitiveType.PrimitiveTypeName.INT32) + .as(OriginalType.DATE) + .named(name); + break; + default: + if (ColumnTypeUtil.isDecimalType(colType)) { + ColumnTypeUtil.DecimalInfo decimalInfo = + ColumnTypeUtil.getDecimalInfo( + colType, PARQUET_DEFAULT_DECIMAL_INFO); + typeBuilder + .optional(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) + .as(OriginalType.DECIMAL) + .precision(decimalInfo.getPrecision()) + .scale(decimalInfo.getScale()) + .length( + Hive3Util.computeMinBytesForPrecision( + decimalInfo.getPrecision())) + .named(name); + + decimalColInfo.put(name, decimalInfo); + } else { + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BINARY).named(name); + } + break; + } + } + + return typeBuilder.named("Pair"); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsTextOutputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsTextOutputFormat.java new file mode 100644 index 0000000000..fd8dc402cc --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsTextOutputFormat.java @@ -0,0 +1,150 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.sink; + +import com.dtstack.chunjun.connector.hive3.enums.CompressType; +import com.dtstack.chunjun.connector.hive3.enums.FileType; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.enums.SizeUnitType; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.WriteRecordException; +import com.dtstack.chunjun.util.ExceptionUtil; + +import org.apache.flink.table.data.RowData; + +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; +import org.apache.hadoop.fs.Path; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; + +/** + * Date: 2021/06/09 Company: www.dtstack.com + * + * @author tudou + */ +public class HdfsTextOutputFormat extends BaseHdfsOutputFormat { + + private static final int NEWLINE = 10; + private transient OutputStream stream; + + @Override + protected void nextBlock() { + super.nextBlock(); + + if (stream != null) { + return; + } + + try { + String currentBlockTmpPath = tmpPath + File.separatorChar + currentFileName; + Path p = new Path(currentBlockTmpPath); + + if (CompressType.TEXT_NONE.equals(compressType)) { + stream = fs.create(p); + } else { + p = new Path(currentBlockTmpPath); + if (compressType == CompressType.TEXT_GZIP) { + stream = new GzipCompressorOutputStream(fs.create(p)); + } else if (compressType == CompressType.TEXT_BZIP2) { + stream = new BZip2CompressorOutputStream(fs.create(p)); + } + } + currentFileIndex++; + LOG.info("subtask:[{}] create block file:{}", taskNumber, currentBlockTmpPath); + } catch (IOException e) { + throw new ChunJunRuntimeException( + Hive3Util.parseErrorMsg(null, ExceptionUtil.getErrorMessage(e)), e); + } + } + + @Override + public void flushDataInternal() { + LOG.info( + "Close current text stream, write data size:[{}]", + SizeUnitType.readableFileSize(bytesWriteCounter.getLocalValue())); + + try { + if (stream != null) { + stream.flush(); + stream.close(); + stream = null; + } + } catch (IOException e) { + throw new ChunJunRuntimeException( + Hive3Util.parseErrorMsg( + "error to flush stream.", ExceptionUtil.getErrorMessage(e)), + e); + } + } + + @Override + @SuppressWarnings("unchecked") + public void writeSingleRecordToFile(RowData rowData) throws WriteRecordException { + if (stream == null) { + nextBlock(); + } + List result = new ArrayList<>(fullColumnNameList.size()); + try { + rowConverter.toExternal(rowData, result); + } catch (Exception e) { + throw new WriteRecordException("can't parse rowData", e, -1, rowData); + } + String line = String.join(hdfsConf.getFieldDelimiter(), result); + + try { + byte[] bytes = line.getBytes(hdfsConf.getEncoding()); + this.stream.write(bytes); + this.stream.write(NEWLINE); + rowsOfCurrentBlock++; + lastRow = rowData; + } catch (IOException e) { + String errorMessage = + Hive3Util.parseErrorMsg( + String.format("writer hdfs error,rowData:{%s}", rowData), + ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(errorMessage, e, -1, rowData); + } + } + + @Override + public void closeSource() { + try { + OutputStream outputStream = this.stream; + if (outputStream != null) { + outputStream.flush(); + this.stream = null; + outputStream.close(); + } + } catch (IOException e) { + throw new ChunJunRuntimeException("close stream error.", e); + } finally { + super.closeSource(); + } + } + + @Override + public CompressType getCompressType() { + return CompressType.getByTypeAndFileType(hdfsConf.getCompress(), FileType.TEXT.name()); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsTransactionOutputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsTransactionOutputFormat.java new file mode 100644 index 0000000000..03312718d8 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsTransactionOutputFormat.java @@ -0,0 +1,360 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.sink; + +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.security.KerberosUtil; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.WriteRecordException; +import com.dtstack.chunjun.util.ExceptionUtil; + +import org.apache.flink.table.data.RowData; + +import org.apache.commons.collections4.MapUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.security.AnnotatedSecurityInfo; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.streaming.HiveStreamingConnection; +import org.apache.hive.streaming.StreamingException; +import org.apache.hive.streaming.StrictDelimitedInputWriter; + +import java.io.IOException; +import java.security.PrivilegedAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** @author liuliu 2022/3/28 */ +public class HdfsTransactionOutputFormat extends HdfsOrcOutputFormat { + + // hive3 + private HiveConf hiveConf; + private StrictDelimitedInputWriter wr; + private HiveStreamingConnection connection; + + public static final String COLUMN_FIELD_DELIMITER = ","; + // 当前事务是否已开启 + private volatile boolean transactionStart; + + @Override + protected void openSource() { + try { + if (hdfsConf.isTransaction()) { + SecurityUtil.setSecurityInfoProviders(new AnnotatedSecurityInfo()); + String currentUser; + try { + currentUser = UserGroupInformation.getCurrentUser().getUserName(); + } catch (IOException e) { + throw new ChunJunRuntimeException(""); + } + Object hadoopUser = hdfsConf.getHadoopConfig().get(HADOOP_USER_NAME); + // 如果配置的 hadoop 用户不为空,那么设置配置中的用户。 + if (hadoopUser != null + && org.apache.commons.lang.StringUtils.isNotEmpty(hadoopUser.toString())) { + currentUser = hadoopUser.toString(); + } + conf = + Hive3Util.getConfiguration( + hdfsConf.getHadoopConfig(), hdfsConf.getDefaultFS()); + try { + // overwrite 需要用到这个 fs 去清空 hdfs 目录. + + fs = + Hive3Util.getFileSystem( + hdfsConf.getHadoopConfig(), + hdfsConf.getDefaultFS(), + currentUser, + null); + } catch (Exception e) { + throw new RuntimeException("Get FileSystem error", e); + } + if (ugi == null) { + getUgi(); + } + ugi.doAs( + (PrivilegedAction) + () -> { + try { + hiveConf = new HiveConf(conf, Hive.class); + if (openKerberos) { + setMetaStoreKerberosConf(); + } + wr = + StrictDelimitedInputWriter.newBuilder() + .withFieldDelimiter(',') + .build(); + } catch (Exception e) { + throw new RuntimeException("init client failed", e); + } + return null; + }); + } + } catch (Exception e) { + throw new ChunJunRuntimeException("{}", e); + } + } + + public void getUgi() throws IOException { + openKerberos = Hive3Util.isOpenKerberos(hdfsConf.getHadoopConfig()); + String currentUser = UserGroupInformation.getCurrentUser().getUserName(); + Object hadoopUser = hdfsConf.getHadoopConfig().get(HADOOP_USER_NAME); + if (hadoopUser != null + && org.apache.commons.lang.StringUtils.isNotEmpty(hadoopUser.toString())) { + currentUser = hadoopUser.toString(); + } + if (openKerberos) { + ugi = Hive3Util.getUGI(hdfsConf.getHadoopConfig(), hdfsConf.getDefaultFS(), null); + LOG.info("user:{}, ", ugi.getShortUserName()); + } else { + ugi = UserGroupInformation.createRemoteUser(currentUser); + } + } + + /** hiveMetaStore 也开启 kerberos 验证,此处设置认证 metastore. */ + private void setMetaStoreKerberosConf() { + String keytabFileName = KerberosUtil.getPrincipalFileName(hdfsConf.getHadoopConfig()); + keytabFileName = KerberosUtil.loadFile(hdfsConf.getHadoopConfig(), keytabFileName, null); + String principal = MapUtils.getString(hdfsConf.getHadoopConfig(), KEY_PRINCIPAL); + String saslEnabled = + MapUtils.getString(hdfsConf.getHadoopConfig(), "hive.metastore.sasl.enabled"); + if (null == saslEnabled) { + saslEnabled = "false"; + } + if (hdfsConf.getHadoopConfig().get("hive.metastore.kerberos.principal") == null) { + hiveConf.setVar(HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL, principal); + } + hiveConf.setVar(HiveConf.ConfVars.METASTORE_KERBEROS_KEYTAB_FILE, keytabFileName); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL, saslEnabled); + } + + @Override + public void writeSingleRecordInternal(RowData rowData) throws WriteRecordException { + String rowString; + try { + rowString = getDataFromRow(rowData); + } catch (Exception e) { + throw new WriteRecordException(String.format("数据写入hdfs异常,row:{%s}", rowData), e); + } + ugi.doAs( + (PrivilegedAction) + () -> { + try { + if (connection == null) { + connection = getConnection(); + } + if (!transactionStart) { + connection.beginTransaction(); + transactionStart = true; + } + connection.write(rowString.getBytes()); + rowsOfCurrentBlock++; + } catch (Exception e) { + throw new RuntimeException( + "writeSingleRecordInternal write data failed, data : " + + rowString, + e); + } + return null; + }); + lastRow = rowData; + lastWriteTime = System.currentTimeMillis(); + } + + private String getDataFromRow(RowData rowData) throws WriteRecordException { + + List list = new ArrayList<>(); + try { + rowConverter.toExternal(rowData, list); + StringBuilder stringBuilder = new StringBuilder(); + for (int i = 0; i < fullColumnNameList.size(); i++) { + Object data = list.get(i); + if (data != null) { + stringBuilder.append(data); + } + if (i != fullColumnNameList.size() - 1) { + stringBuilder.append(COLUMN_FIELD_DELIMITER); + } + } + return stringBuilder.toString(); + } catch (Exception e) { + String errorMessage = + Hive3Util.parseErrorMsg( + String.format("writer hdfs error,rowData:{%s}", rowData), + ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(errorMessage, e, -1, rowData); + } + } + + @Override + protected void writeMultipleRecordsInternal() { + // hive3 事务表 + if (hdfsConf.isTransaction()) { + ugi.doAs( + (PrivilegedAction) + () -> { + try { + if (connection == null) { + connection = getConnection(); + } + if (!transactionStart) { + connection.beginTransaction(); + transactionStart = true; + } + String rowString; + for (RowData rowData : rows) { + rowString = getDataFromRow(rowData); + connection.write(rowString.getBytes()); + } + if (rows != null && rows.size() > 0) { + // 取最后一条数据 + lastRow = rows.get(rows.size() - 1); + rowsOfCurrentBlock += rows.size(); + lastWriteTime = System.currentTimeMillis(); + } + } catch (Throwable e) { + LOG.error("writeMultipleRecordsInternal 方法, 写入多条数据失败", e); + throw new RuntimeException("WRITER DATA ERROR", e); + } + return null; + }); + } + } + + private HiveStreamingConnection getConnection() { + HiveStreamingConnection.Builder builder = + HiveStreamingConnection.newBuilder() + .withDatabase(hdfsConf.getSchema()) + .withTable(hdfsConf.getTable()) + .withAgentInfo("chunjun3_hive3writer_" + Thread.currentThread().getName()) + .withTransactionBatchSize(hdfsConf.getBatchSize()) // 何时触发压缩 + .withRecordWriter(wr) + .withHiveConf(hiveConf); + if (StringUtils.isNotEmpty(hdfsConf.getPartitionName())) { + String[] multiPartitionName = {hdfsConf.getPartitionName()}; + if (hdfsConf.getPartitionName().contains("/")) { + multiPartitionName = hdfsConf.getPartitionName().split("/"); + } + List partitions = Arrays.asList(multiPartitionName); + builder.withStaticPartitionValues(partitions); + } + HiveStreamingConnection connect; + try { + connect = builder.connect(); + } catch (Throwable e) { // Exception 有可能捕获不到 + LOG.error("HdfsOrcOutputFormat getConnection 方法, 连接 Hive Metastore 失败", e); + String metastoreUri = hiveConf.get("hive.metastore.uris"); + throw new RuntimeException( + "Error connecting to Hive Metastore URI: " + + metastoreUri + + ". " + + e.getMessage(), + e); + } + return connect; + } + + @Override + protected void checkOutputDir() { + // hive3 transaction table,doNothing + } + + @Override + protected void deleteTmpDataDir() { + // hive3 transaction table,doNothing + } + + @Override + protected void preCommit() { + snapshotWriteCounter.add(rowsOfCurrentBlock); + rowsOfCurrentBlock = 0; + formatState.setJobId(jobId); + } + + @Override + public void commit(long checkpointId) { + if (connection != null && transactionStart) { + try { + connection.commitTransaction(); + } catch (StreamingException e) { + try { + connection.abortTransaction(); + } catch (StreamingException ex) { + throw new ChunJunRuntimeException("hive3 abort transaction failed"); + } + throw new ChunJunRuntimeException("hive3 commit transaction failed"); + } finally { + transactionStart = false; + } + } + } + + @Override + public void rollback(long checkpointId) { + try { + if (connection != null && transactionStart) { + connection.abortTransaction(); + } + } catch (StreamingException e) { + throw new ChunJunRuntimeException("hive3 commit transaction failed"); + } finally { + transactionStart = false; + } + } + + @Override + public void closeInternal() { + snapshotWriteCounter.add(rowsOfCurrentBlock); + rowsOfCurrentBlock = 0; + closeSource(); + } + + @Override + protected void closeSource() { + ugi.doAs( + (PrivilegedAction) + () -> { + if (connection != null) { + try { + connection.commitTransaction(); + } catch (StreamingException e) { + throw new ChunJunRuntimeException( + "hive3 commit transaction failed"); + } + } + if (connection != null) { + connection.close(); + connection = null; + } + return null; + }); + } + + @Override + public void finalizeGlobal(int parallelism) { + // hive3 transaction table,doNothing + } + + @Override + protected void deleteDataDir() { + deleteDirectory(outputFilePath, false); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/Hive3OutputFormatBuilder.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/Hive3OutputFormatBuilder.java new file mode 100644 index 0000000000..a1e07662e1 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/Hive3OutputFormatBuilder.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.sink; + +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.enums.FileType; +import com.dtstack.chunjun.sink.format.FileOutputFormatBuilder; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; + +/** @author liuliu 2022/3/23 */ +public class Hive3OutputFormatBuilder extends FileOutputFormatBuilder { + BaseHdfsOutputFormat outputFormat; + + public Hive3OutputFormatBuilder(String fileType, boolean isHiveTransactionTable) { + switch (FileType.getByName(fileType)) { + case ORC: + if (isHiveTransactionTable) { + outputFormat = new HdfsTransactionOutputFormat(); + } else { + outputFormat = new HdfsOrcOutputFormat(); + } + break; + case PARQUET: + outputFormat = new HdfsParquetOutputFormat(); + break; + case TEXT: + outputFormat = new HdfsTextOutputFormat(); + break; + default: + throw new UnsupportedTypeException(fileType); + } + super.setFormat(outputFormat); + } + + public void setHdfsConf(HdfsConf hdfsConf) { + super.setBaseFileConf(hdfsConf); + outputFormat.setHdfsConf(hdfsConf); + } + + @Override + protected void checkFormat() {} +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/Hive3SinkFactory.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/Hive3SinkFactory.java new file mode 100644 index 0000000000..43576dc063 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/Hive3SinkFactory.java @@ -0,0 +1,114 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.sink; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.conf.SyncConf; +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.converter.HdfsRawTypeConverter; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.RawTypeConverter; +import com.dtstack.chunjun.sink.SinkFactory; +import com.dtstack.chunjun.util.GsonUtil; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.data.RowData; + +import org.apache.commons.collections.CollectionUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +/** @author liuliu 2022/3/23 */ +public class Hive3SinkFactory extends SinkFactory { + private final HdfsConf hdfsConf; + + public Hive3SinkFactory(SyncConf syncConf) { + super(syncConf); + hdfsConf = + GsonUtil.GSON.fromJson( + GsonUtil.GSON.toJson(syncConf.getWriter().getParameter()), HdfsConf.class); + hdfsConf.setColumn(syncConf.getWriter().getFieldList()); + hdfsConf.setFieldDelimiter( + com.dtstack.chunjun.util.StringUtil.convertRegularExpr( + hdfsConf.getFieldDelimiter())); + + initFullColumnMessage(hdfsConf); + super.initCommonConf(hdfsConf); + } + + @Override + public RawTypeConverter getRawTypeConverter() { + return HdfsRawTypeConverter::apply; + } + + @Override + public DataStreamSink createSink(DataStream dataSet) { + Hive3OutputFormatBuilder builder = + new Hive3OutputFormatBuilder(hdfsConf.getFileType(), hdfsConf.isTransaction()); + builder.setHdfsConf(hdfsConf); + AbstractRowConverter rowConverter = + Hive3Util.createRowConverter( + useAbstractBaseColumn, + hdfsConf.getFileType(), + hdfsConf.getColumn(), + getRawTypeConverter(), + hdfsConf); + + builder.setRowConverter(rowConverter, useAbstractBaseColumn); + return createOutput(dataSet, builder.finish()); + } + + public void initFullColumnMessage(HdfsConf hdfsConf) { + List fullColumnNameList = hdfsConf.getFullColumnName(); + List fullColumnTypeList = hdfsConf.getFullColumnType(); + if (CollectionUtils.isEmpty(fullColumnNameList)) { + fullColumnNameList = + hdfsConf.getColumn().stream() + .map(FieldConf::getName) + .collect(Collectors.toList()); + } + if (CollectionUtils.isEmpty(fullColumnTypeList)) { + fullColumnTypeList = new ArrayList<>(fullColumnNameList.size()); + } + int[] fullColumnIndexes = new int[fullColumnNameList.size()]; + List fieldConfList = hdfsConf.getColumn(); + for (int i = 0; i < fullColumnNameList.size(); i++) { + String columnName = fullColumnNameList.get(i); + int j = 0; + for (; j < fieldConfList.size(); j++) { + FieldConf fieldConf = fieldConfList.get(j); + if (columnName.equalsIgnoreCase(fieldConf.getName())) { + fullColumnTypeList.add(fieldConf.getType()); + break; + } + } + if (j == fieldConfList.size()) { + j = -1; + } + fullColumnIndexes[i] = j; + } + hdfsConf.setFullColumnName(fullColumnNameList); + hdfsConf.setFullColumnType(fullColumnTypeList); + hdfsConf.setFullColumnIndexes(fullColumnIndexes); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/BaseHdfsInputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/BaseHdfsInputFormat.java new file mode 100644 index 0000000000..efc2b69be3 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/BaseHdfsInputFormat.java @@ -0,0 +1,155 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.source.format.BaseRichInputFormat; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.util.PluginUtil; + +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.core.io.InputSplit; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.security.UserGroupInformation; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.HashMap; +import java.util.Map; + +/** @author liuliu 2022/3/23 */ +public abstract class BaseHdfsInputFormat extends BaseRichInputFormat { + + protected HdfsConf hdfsConf; + + /** the key to read data into */ + protected Object key; + /** the value to read data into */ + protected Object value; + + protected boolean openKerberos; + + protected transient UserGroupInformation ugi; + protected transient org.apache.hadoop.mapred.InputFormat inputFormat; + protected transient JobConf hadoopJobConf; + protected transient RecordReader recordReader; + + @Override + protected InputSplit[] createInputSplitsInternal(int minNumSplits) throws Exception { + if (Hive3Util.isOpenKerberos(hdfsConf.getHadoopConfig())) { + DistributedCache distributedCache = + PluginUtil.createDistributedCacheFromContextClassLoader(); + UserGroupInformation ugi = + Hive3Util.getUGI( + hdfsConf.getHadoopConfig(), hdfsConf.getDefaultFS(), distributedCache); + return ugi.doAs( + (PrivilegedExceptionAction) + () -> { + try { + return createHdfsSplit(minNumSplits); + } catch (Exception e) { + throw new ChunJunRuntimeException( + "error to create hdfs splits", e); + } + }); + } else { + return createHdfsSplit(minNumSplits); + } + } + + /** init Hadoop Job Config */ + protected void initHadoopJobConf() { + hadoopJobConf = Hive3Util.getJobConf(hdfsConf.getHadoopConfig(), hdfsConf.getDefaultFS()); + hadoopJobConf.set(HdfsPathFilter.KEY_REGEX, hdfsConf.getFilterRegex()); + Hive3Util.setHadoopUserName(hadoopJobConf); + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + initHadoopJobConf(); + this.inputFormat = createMapredInputFormat(); + openKerberos = Hive3Util.isOpenKerberos(hdfsConf.getHadoopConfig()); + if (openKerberos) { + ugi = + Hive3Util.getUGI( + hdfsConf.getHadoopConfig(), + hdfsConf.getDefaultFS(), + getRuntimeContext().getDistributedCache()); + } + } + + protected abstract InputSplit[] createHdfsSplit(int minNumSplits) throws IOException; + + @Override + @SuppressWarnings("unchecked") + public boolean reachedEnd() throws IOException { + return !recordReader.next(key, value); + } + + @Override + public void closeInternal() throws IOException { + if (recordReader != null) { + recordReader.close(); + } + } + + /** + * 从hdfs路径中获取当前分区信息 + * + * @param path hdfs路径 + */ + public void findCurrentPartition(Path path) { + if (null == path) { + LOG.warn("The Path finding partition value is null"); + return; + } + Map partitionAndValueMap = new HashMap<>(16); + String fileParentPath = path.getParent().toString(); + // 为了给下面分区列找到分区字段,将文件的父路径切分,缓存到 map 中。 + final String[] pathNodes = fileParentPath.split("/"); + for (String pathNode : pathNodes) { + if (pathNode.contains("=") && pathNode.length() >= 3) { + final String[] partitionAndValue = pathNode.split("="); + // 分区 和 分区值 放入 map 中, eg : pt=20210906 + if (partitionAndValue.length == 2) { + partitionAndValueMap.put(partitionAndValue[0], partitionAndValue[1]); + } + } + } + // 从 map 里面找出对应分区字段,然后给该列设置值。 + for (FieldConf fieldConf : hdfsConf.getColumn()) { + // 如果此列是分区字段 + if (fieldConf.getPart()) { + fieldConf.setValue(partitionAndValueMap.get(fieldConf.getName())); + } + } + } + + public abstract org.apache.hadoop.mapred.InputFormat createMapredInputFormat(); + + public void sethdfsConf(HdfsConf hdfsConf) { + this.hdfsConf = hdfsConf; + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsOrcInputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsOrcInputFormat.java new file mode 100644 index 0000000000..3f9e3cee9c --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsOrcInputFormat.java @@ -0,0 +1,260 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.inputSplit.HdfsOrcInputSplit; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.ReadRecordException; + +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.orc.OrcFile; +import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; +import org.apache.hadoop.hive.ql.io.orc.OrcSplit; +import org.apache.hadoop.hive.ql.io.orc.OrcStruct; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.Reporter; + +import java.io.IOException; +import java.security.PrivilegedAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +/** @author liuliu 2022/3/23 */ +public class HdfsOrcInputFormat extends BaseHdfsInputFormat { + protected transient FileSystem fs; + + protected transient String[] fullColNames; + + protected transient StructObjectInspector inspector; + + protected transient List fields; + + protected static final String COMPLEX_FIELD_TYPE_SYMBOL_REGEX = ".*(<|>|\\{|}|[|]).*"; + + private final AtomicBoolean isInit = new AtomicBoolean(false); + + @Override + protected InputSplit[] createHdfsSplit(int minNumSplits) throws IOException { + initHadoopJobConf(); + // 非事务表创建分片 + org.apache.hadoop.mapred.FileInputFormat.setInputPaths(hadoopJobConf, hdfsConf.getPath()); + org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter( + hadoopJobConf, HdfsPathFilter.class); + + OrcInputFormat orcInputFormat = new OrcInputFormat(); + org.apache.hadoop.mapred.InputSplit[] splits = + orcInputFormat.getSplits(hadoopJobConf, minNumSplits); + + if (splits != null) { + List list = new ArrayList<>(splits.length); + int i = 0; + for (org.apache.hadoop.mapred.InputSplit split : splits) { + OrcSplit orcSplit = (OrcSplit) split; + if (orcSplit.getLength() > 49) { + list.add(new HdfsOrcInputSplit(orcSplit, i)); + i++; + } + } + return list.toArray(new HdfsOrcInputSplit[i]); + } + return null; + } + + @Override + public InputFormat createMapredInputFormat() { + return new OrcInputFormat(); + } + + @Override + protected void openInternal(InputSplit inputSplit) throws IOException { + if (super.openKerberos) { + ugi.doAs( + (PrivilegedAction) + () -> { + try { + orcOpenInternal(inputSplit); + } catch (IOException e) { + throw new ChunJunRuntimeException( + "failed to open orc internal", e); + } + return null; + }); + } else { + orcOpenInternal(inputSplit); + } + } + + protected void orcOpenInternal(InputSplit inputSplit) throws IOException { + OrcSplit orcSplit = ((HdfsOrcInputSplit) inputSplit).getOrcSplit(); + fs = FileSystem.get(hadoopJobConf); + init(orcSplit); + openOrcReader(inputSplit); + } + + @Override + protected RowData nextRecordInternal(RowData rowData) throws ReadRecordException { + try { + List fieldConfList = hdfsConf.getColumn(); + GenericRowData genericRowData = + new GenericRowData(Math.max(fieldConfList.size(), fullColNames.length)); + if (fieldConfList.size() == 1 + && ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName())) { + + for (int i = 0; i < fullColNames.length; i++) { + Object col = inspector.getStructFieldData(value, fields.get(i)); + if (col != null) { + col = Hive3Util.getWritableValue(col); + } + genericRowData.setField(i, col); + } + } else { + for (int i = 0; i < fieldConfList.size(); i++) { + FieldConf fieldConf = fieldConfList.get(i); + Object val = null; + if (fieldConf.getValue() != null) { + val = fieldConf.getValue(); + } else if (fieldConf.getIndex() != -1) { + val = inspector.getStructFieldData(value, fields.get(fieldConf.getIndex())); + } + + if (val != null) { + val = Hive3Util.getWritableValue(val); + } + + genericRowData.setField(i, val); + } + } + return rowConverter.toInternal(genericRowData); + } catch (Exception e) { + throw new ReadRecordException("", e, 0, rowData); + } + } + + protected List parseColumnAndType(String typeStruct) { + List cols = new ArrayList<>(); + List splits = Arrays.asList(typeStruct.split(",")); + Iterator it = splits.iterator(); + while (it.hasNext()) { + StringBuilder current = new StringBuilder(it.next()); + if (!current.toString().contains("(") && !current.toString().contains(")")) { + cols.add(current.toString()); + continue; + } + + if (current.toString().contains("(") && current.toString().contains(")")) { + cols.add(current.toString()); + continue; + } + + if (current.toString().contains("(") && !current.toString().contains(")")) { + while (it.hasNext()) { + String next = it.next(); + current.append(",").append(next); + if (next.contains(")")) { + break; + } + } + + cols.add(current.toString()); + } + } + + return cols; + } + + public void init(OrcSplit orcSplit) throws IOException { + try { + if (!isInit.get()) { + init(orcSplit.getPath()); + isInit.set(true); + } + } catch (Exception e) { + throw new IOException("init [inspector] error", e); + } + } + + private void openOrcReader(InputSplit inputSplit) throws IOException { + numReadCounter = getRuntimeContext().getLongCounter("numRead"); + HdfsOrcInputSplit hdfsOrcInputSplit = (HdfsOrcInputSplit) inputSplit; + OrcSplit orcSplit = hdfsOrcInputSplit.getOrcSplit(); + findCurrentPartition(orcSplit.getPath()); + recordReader = inputFormat.getRecordReader(orcSplit, hadoopJobConf, Reporter.NULL); + key = recordReader.createKey(); + value = recordReader.createValue(); + fields = inspector.getAllStructFieldRefs(); + } + + private void init(Path path) throws Exception { + OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(hadoopJobConf); + readerOptions.filesystem(fs); + + org.apache.hadoop.hive.ql.io.orc.Reader reader = OrcFile.createReader(path, readerOptions); + String typeStruct = reader.getObjectInspector().getTypeName(); + // struct + if (StringUtils.isEmpty(typeStruct)) { + throw new RuntimeException("can't retrieve type struct from " + path); + } + + int startIndex = typeStruct.indexOf("<") + 1; + int endIndex = typeStruct.lastIndexOf(">"); + typeStruct = typeStruct.substring(startIndex, endIndex); + + if (typeStruct.matches(COMPLEX_FIELD_TYPE_SYMBOL_REGEX)) { + throw new RuntimeException( + "Field types such as array, map, and struct are not supported."); + } + + List cols = parseColumnAndType(typeStruct); + + fullColNames = new String[cols.size()]; + String[] fullColTypes = new String[cols.size()]; + + for (int i = 0; i < cols.size(); ++i) { + String[] temp = cols.get(i).split(":"); + fullColNames[i] = temp[0]; + fullColTypes[i] = temp[1]; + } + + Properties p = new Properties(); + p.setProperty("columns", StringUtils.join(fullColNames, ",")); + p.setProperty("columns.types", StringUtils.join(fullColTypes, ":")); + + OrcSerde orcSerde = new OrcSerde(); + orcSerde.initialize(hadoopJobConf, p); + + this.inspector = (StructObjectInspector) orcSerde.getObjectInspector(); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsParquetInputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsParquetInputFormat.java new file mode 100644 index 0000000000..9e9617e688 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsParquetInputFormat.java @@ -0,0 +1,393 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.inputSplit.HdfsParquetInputSplit; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.enums.ColumnType; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.ReadRecordException; + +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; + +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.example.GroupReadSupport; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.DecimalMetadata; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.security.PrivilegedAction; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** @author liuliu 2022/3/23 */ +public class HdfsParquetInputFormat extends BaseHdfsInputFormat { + + private List currentSplitFilePaths; + private static final int JULIAN_EPOCH_OFFSET_DAYS = 2440588; + private static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1); + private static final long NANOS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1); + private static final int TIMESTAMP_BINARY_LENGTH = 12; + + private transient Group currentLine; + private transient ParquetReader currentFileReader; + private transient List fullColNames; + private transient List fullColTypes; + private transient int currentFileIndex = 0; + + @Override + protected InputSplit[] createHdfsSplit(int minNumSplits) { + initHadoopJobConf(); + Set allFilePaths; + HdfsPathFilter pathFilter = new HdfsPathFilter(hdfsConf.getFilterRegex()); + try (FileSystem fs = FileSystem.get(hadoopJobConf)) { + allFilePaths = Hive3Util.getAllPartitionPath(hdfsConf.getPath(), fs, pathFilter); + } catch (Exception e) { + throw new ChunJunRuntimeException("failed to get parquet file path", e); + } + + if (allFilePaths.size() > 0) { + HdfsParquetInputSplit[] splits = new HdfsParquetInputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new HdfsParquetInputSplit(i, new ArrayList<>()); + } + + Iterator it = allFilePaths.iterator(); + while (it.hasNext()) { + for (HdfsParquetInputSplit split : splits) { + if (it.hasNext()) { + split.getPaths().add(it.next()); + } + } + } + + return splits; + } + + return new HdfsParquetInputSplit[0]; + } + + @Override + public InputFormat createMapredInputFormat() { + return null; + } + + @Override + protected void openInternal(InputSplit inputSplit) { + currentSplitFilePaths = ((HdfsParquetInputSplit) inputSplit).getPaths(); + } + + @Override + protected RowData nextRecordInternal(RowData rowData) throws ReadRecordException { + List fieldConfList = hdfsConf.getColumn(); + GenericRowData genericRowData; + if (fieldConfList.size() == 1 + && ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName())) { + genericRowData = new GenericRowData(fullColNames.size()); + for (int i = 0; i < fullColNames.size(); i++) { + Object obj = getData(currentLine, fullColTypes.get(i), i); + genericRowData.setField(i, obj); + } + } else { + genericRowData = + new GenericRowData(Math.max(fieldConfList.size(), fullColNames.size())); + for (int i = 0; i < fieldConfList.size(); i++) { + FieldConf fieldConf = fieldConfList.get(i); + Object obj = null; + if (fieldConf.getValue() != null) { + obj = fieldConf.getValue(); + } else if (fieldConf.getIndex() != null + && fieldConf.getIndex() < fullColNames.size()) { + if (currentLine.getFieldRepetitionCount(fieldConf.getIndex()) > 0) { + obj = getData(currentLine, fieldConf.getType(), fieldConf.getIndex()); + } + } + genericRowData.setField(i, obj); + } + } + + try { + return rowConverter.toInternal(genericRowData); + } catch (Exception e) { + throw new ReadRecordException("", e, 0, rowData); + } + } + + /** @return millisecond */ + private long getTimestampMillis(Binary timestampBinary) { + if (timestampBinary.length() != TIMESTAMP_BINARY_LENGTH) { + return 0; + } + + byte[] bytes = timestampBinary.getBytes(); + + long timeOfDayNanos = + Longs.fromBytes( + bytes[7], bytes[6], bytes[5], bytes[4], bytes[3], bytes[2], bytes[1], + bytes[0]); + int julianDay = Ints.fromBytes(bytes[11], bytes[10], bytes[9], bytes[8]); + + return julianDayToMillis(julianDay) + (timeOfDayNanos / NANOS_PER_MILLISECOND); + } + + private long julianDayToMillis(int julianDay) { + return (julianDay - JULIAN_EPOCH_OFFSET_DAYS) * MILLIS_IN_DAY; + } + + private BigDecimal longToDecimalStr(long value, int scale) { + BigInteger bi = BigInteger.valueOf(value); + return new BigDecimal(bi, scale); + } + + private BigDecimal binaryToDecimalStr(Binary binary, int scale) { + BigInteger bi = new BigInteger(binary.getBytes()); + return new BigDecimal(bi, scale); + } + + public Object getData(Group currentLine, String type, int index) { + Object data = null; + ColumnType columnType = ColumnType.fromString(type); + + try { + if (index == -1) { + return null; + } + + Type colSchemaType = currentLine.getType().getType(index); + switch (columnType.name().toLowerCase(Locale.ENGLISH)) { + case "tinyint": + case "smallint": + case "int": + data = currentLine.getInteger(index, 0); + break; + case "bigint": + data = currentLine.getLong(index, 0); + break; + case "float": + data = currentLine.getFloat(index, 0); + break; + case "double": + data = currentLine.getDouble(index, 0); + break; + case "binary": + Binary binaryData = currentLine.getBinary(index, 0); + data = binaryData.getBytes(); + break; + case "char": + case "varchar": + case "string": + data = currentLine.getString(index, 0); + break; + case "boolean": + data = currentLine.getBoolean(index, 0); + break; + case "timestamp": + { + long time = getTimestampMillis(currentLine.getInt96(index, 0)); + data = new Timestamp(time); + break; + } + case "decimal": + { + DecimalMetadata dm = ((PrimitiveType) colSchemaType).getDecimalMetadata(); + String primitiveTypeName = + currentLine + .getType() + .getType(index) + .asPrimitiveType() + .getPrimitiveTypeName() + .name(); + if (ColumnType.INT32.name().equals(primitiveTypeName)) { + int intVal = currentLine.getInteger(index, 0); + data = longToDecimalStr(intVal, dm.getScale()); + } else if (ColumnType.INT64.name().equals(primitiveTypeName)) { + long longVal = currentLine.getLong(index, 0); + data = longToDecimalStr(longVal, dm.getScale()); + } else { + Binary binary = currentLine.getBinary(index, 0); + data = binaryToDecimalStr(binary, dm.getScale()); + } + break; + } + case "date": + { + String val = currentLine.getValueToString(index, 0); + data = + new Timestamp(Integer.parseInt(val) * MILLIS_IN_DAY) + .toString() + .substring(0, 10); + break; + } + default: + data = currentLine.getValueToString(index, 0); + break; + } + } catch (Exception e) { + LOG.error("error to get data from parquet group.", e); + } + + return data; + } + + @Override + public boolean reachedEnd() throws IOException { + return !nextLine(); + } + + private void getNextLine() throws IOException { + if (currentFileReader != null) { + if (openKerberos) { + ugi.doAs( + (PrivilegedAction) + () -> { + try { + currentLine = currentFileReader.read(); + } catch (IOException e) { + throw new ChunJunRuntimeException( + "failed to read parquet data with kerberos"); + } + return null; + }); + } else { + currentLine = currentFileReader.read(); + } + } + } + + private boolean nextLine() throws IOException { + getNextLine(); + if (currentLine != null) { + setMetaColumns(); + return true; + } + while (currentFileIndex <= currentSplitFilePaths.size() - 1) { + if (openKerberos) { + ugi.doAs( + (PrivilegedAction) + () -> { + try { + nextFile(); + return null; + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } else { + nextFile(); + } + getNextLine(); + if (currentLine != null) { + setMetaColumns(); + return true; + } + } + return false; + } + + private void nextFile() throws IOException { + Path path = new Path(currentSplitFilePaths.get(currentFileIndex)); + findCurrentPartition(path); + ParquetReader.Builder reader = + ParquetReader.builder(new GroupReadSupport(), path).withConf(hadoopJobConf); + currentFileReader = reader.build(); + currentFileIndex++; + } + + public void setMetaColumns() { + if (fullColNames == null && currentLine != null) { + fullColNames = new ArrayList<>(); + fullColTypes = new ArrayList<>(); + List types = currentLine.getType().getFields(); + for (Type type : types) { + fullColNames.add(type.getName().toUpperCase()); + fullColTypes.add( + getTypeName(type.asPrimitiveType().getPrimitiveTypeName().getMethod)); + } + + for (FieldConf fieldConf : hdfsConf.getColumn()) { + String name = fieldConf.getName(); + if (StringUtils.isNotBlank(name)) { + name = name.toUpperCase(); + if (fullColNames.contains(name)) { + fieldConf.setIndex(fullColNames.indexOf(name)); + } else { + fieldConf.setIndex(-1); + } + } + } + } + } + + private String getTypeName(String method) { + String typeName; + switch (method) { + case "getBoolean": + case "getInteger": + typeName = "int"; + break; + case "getInt96": + typeName = "bigint"; + break; + case "getFloat": + typeName = "float"; + break; + case "getDouble": + typeName = "double"; + break; + case "getBinary": + typeName = "binary"; + break; + default: + typeName = "string"; + } + + return typeName; + } + + @Override + public void closeInternal() throws IOException { + if (currentFileReader != null) { + currentFileReader.close(); + currentFileReader = null; + } + + currentLine = null; + currentFileIndex = 0; + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsPathFilter.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsPathFilter.java new file mode 100644 index 0000000000..27bf7f74d8 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsPathFilter.java @@ -0,0 +1,72 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.source; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; + +import java.util.regex.Pattern; + +/** + * Date: 2021/06/08 Company: www.dtstack.com + * + * @author tudou + */ +public class HdfsPathFilter implements PathFilter, JobConfigurable { + + public static final String KEY_REGEX = "file.path.regexFilter"; + private static final String DEFAULT_REGEX = ".*"; + private static final PathFilter HIDDEN_FILE_FILTER = + p -> { + String name = p.getName(); + return !name.startsWith("_") && !name.startsWith("."); + }; + private static Pattern PATTERN; + private String regex; + + public HdfsPathFilter() {} + + public HdfsPathFilter(String regex) { + this.regex = regex; + compileRegex(); + } + + @Override + public boolean accept(Path path) { + if (!HIDDEN_FILE_FILTER.accept(path)) { + return false; + } + + return PATTERN.matcher(path.getName()).matches(); + } + + @Override + public void configure(JobConf jobConf) { + this.regex = jobConf.get(KEY_REGEX); + compileRegex(); + } + + /** compile regex */ + private void compileRegex() { + String compileRegex = StringUtils.isEmpty(regex) ? DEFAULT_REGEX : regex; + PATTERN = Pattern.compile(compileRegex); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsTextInputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsTextInputFormat.java new file mode 100644 index 0000000000..e4aa7d0b4d --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsTextInputFormat.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.hive3.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.inputSplit.HdfsTextInputSplit; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.ReadRecordException; + +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.TextInputFormat; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.security.PrivilegedAction; +import java.util.List; + +/** @author liuliu 2022/3/23 */ +public class HdfsTextInputFormat extends BaseHdfsInputFormat { + + @Override + protected InputSplit[] createHdfsSplit(int minNumSplits) throws IOException { + initHadoopJobConf(); + // 是否在MapReduce中递归遍历Input目录 + hadoopJobConf.set("mapreduce.input.fileinputformat.input.dir.recursive", "true"); + FileInputFormat.setInputPathFilter(hadoopJobConf, HdfsPathFilter.class); + FileInputFormat.setInputPaths(hadoopJobConf, hdfsConf.getPath()); + TextInputFormat inputFormat = new TextInputFormat(); + inputFormat.configure(hadoopJobConf); + org.apache.hadoop.mapred.InputSplit[] splits; + try { + splits = inputFormat.getSplits(hadoopJobConf, minNumSplits); + } catch (IOException e) { + throw new ChunJunRuntimeException("failed to get hdfs text splits", e); + } + if (splits != null) { + HdfsTextInputSplit[] hdfsTextInputSplits = new HdfsTextInputSplit[splits.length]; + for (int i = 0; i < splits.length; ++i) { + hdfsTextInputSplits[i] = new HdfsTextInputSplit(splits[i], i); + } + return hdfsTextInputSplits; + } + return null; + } + + @Override + protected void openInternal(InputSplit inputSplit) { + if (super.openKerberos) { + ugi.doAs( + (PrivilegedAction) + () -> { + try { + initHdfsTextReader(inputSplit); + } catch (Exception e) { + throw new ChunJunRuntimeException( + "error to open Internal, split = " + inputSplit, e); + } + return null; + }); + } else { + initHdfsTextReader(inputSplit); + } + } + + private void initHdfsTextReader(InputSplit inputSplit) { + try { + HdfsTextInputSplit hdfsTextInputSplit = (HdfsTextInputSplit) inputSplit; + org.apache.hadoop.mapred.InputSplit fileSplit = hdfsTextInputSplit.getTextSplit(); + findCurrentPartition(((FileSplit) fileSplit).getPath()); + recordReader = inputFormat.getRecordReader(fileSplit, hadoopJobConf, Reporter.NULL); + key = new LongWritable(); + value = new Text(); + } catch (Exception e) { + throw new ChunJunRuntimeException("error to open Internal, split = " + inputSplit, e); + } + } + + @Override + public InputFormat createMapredInputFormat() { + return new TextInputFormat(); + } + + @Override + protected RowData nextRecordInternal(RowData rowData) throws ReadRecordException { + try { + List fieldConfList = hdfsConf.getColumn(); + String line = + new String( + ((Text) value).getBytes(), + 0, + ((Text) value).getLength(), + StandardCharsets.UTF_8); + String[] fields = + StringUtils.splitPreserveAllTokens(line, hdfsConf.getFieldDelimiter()); + GenericRowData genericRowData = + new GenericRowData(Math.max(fields.length, fieldConfList.size())); + if (fieldConfList.size() == 1 + && ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName())) { + for (int i = 0; i < fields.length; i++) { + genericRowData.setField(i, fields[i]); + } + } else { + for (int i = 0; i < fieldConfList.size(); i++) { + FieldConf fieldConf = fieldConfList.get(i); + + Object value = null; + if (fieldConf.getValue() != null) { + value = fieldConf.getValue(); + } else if (fieldConf.getIndex() != null + && fieldConf.getIndex() < fields.length) { + String strVal = fields[fieldConf.getIndex()]; + if (!Hive3Util.NULL_VALUE.equals(strVal)) { + value = strVal; + } + } + genericRowData.setField(i, value); + } + } + rowData = rowConverter.toInternal(genericRowData); + } catch (Exception e) { + throw new ReadRecordException("", e, 0, rowData); + } + return rowData; + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsTransactionInputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsTransactionInputFormat.java new file mode 100644 index 0000000000..58b24e46e8 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/HdfsTransactionInputFormat.java @@ -0,0 +1,207 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.source; + +import com.dtstack.chunjun.connector.hive3.inputSplit.HdfsOrcInputSplit; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.util.ExceptionUtil; + +import org.apache.flink.core.io.InputSplit; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.io.IOConstants; +import org.apache.hadoop.hive.ql.io.orc.OrcFile; +import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; +import org.apache.hadoop.hive.ql.io.orc.OrcSplit; +import org.apache.hadoop.hive.ql.io.orc.OrcStruct; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.Reporter; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Properties; +import java.util.Set; + +/** @author liuliu 2022/3/28 */ +public class HdfsTransactionInputFormat extends HdfsOrcInputFormat { + @Override + protected InputSplit[] createHdfsSplit(int minNumSplits) { + LOG.info("To read hive transaction table, create OrcSplit."); + hadoopJobConf = Hive3Util.getJobConf(hdfsConf.getHadoopConfig(), hdfsConf.getDefaultFS()); + // hive3 事务表必须设置的属性 + hadoopJobConf.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); + // 此处防止 shaded 后,文件系统改变,导致错误。 + hadoopJobConf.set( + "fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + hadoopJobConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // 最大写事务 id + hadoopJobConf.set("hive.txn.valid.txns", Long.MAX_VALUE + ":0"); + FileSystem fileSystem; + try { + fileSystem = + Hive3Util.getFileSystem( + hdfsConf.getHadoopConfig(), hdfsConf.getDefaultFS(), null); + // 递归找到所有分区路径 + Set allPartitionPath = + Hive3Util.getAllPartitionPath( + hdfsConf.getPath(), + fileSystem, + new HdfsPathFilter(hdfsConf.getFilterRegex())); + // 每个分区路径 + LinkedList allSplit = new LinkedList<>(); + int splitNumber = 0; + for (String partitionPath : allPartitionPath) { + OrcInputFormat orcInputFormat = new OrcInputFormat(); + // 每次按照分区的路径传入,然后获取分片,内部会按 acid 方式解析。分区的上一级传入,无法自动解析。 + hadoopJobConf.set("mapred.input.dir", partitionPath); + // 每个分区获取一个分片 + org.apache.hadoop.mapred.InputSplit[] inputSplits = + orcInputFormat.getSplits(hadoopJobConf, minNumSplits); + // 转成 HdfsOrcInputSplit 为了 JM 到 TM 序列化。 + for (org.apache.hadoop.mapred.InputSplit inputSplit : inputSplits) { + OrcSplit orcSplit = (OrcSplit) inputSplit; + if (orcSplit.getLength() > 49) { + allSplit.add(new HdfsOrcInputSplit(orcSplit, splitNumber)); + splitNumber++; + } + } + } + return allSplit.toArray(new HdfsOrcInputSplit[0]); + } catch (Exception e) { + LOG.error("hive3 transaction table create split error", e); + throw new RuntimeException(e); + } + } + + protected void orcOpenInternal(InputSplit inputSplit) throws IOException { + fs = FileSystem.get(hadoopJobConf); + openAcidRecordReader(inputSplit); + } + + /** hive 事务表创建 RecordReader */ + protected void openAcidRecordReader(InputSplit inputSplit) { + numReadCounter = getRuntimeContext().getLongCounter("numRead"); + HdfsOrcInputSplit hdfsOrcInputSplit = (HdfsOrcInputSplit) inputSplit; + OrcSplit orcSplit = null; + try { + orcSplit = hdfsOrcInputSplit.getOrcSplit(); + } catch (IOException e) { + LOG.error( + "Get orc split error, hdfsOrcInputSplit = {}, Exception = {}", + hdfsOrcInputSplit, + ExceptionUtil.getErrorMessage(e)); + } + Path path = null; + if (orcSplit != null) { + path = orcSplit.getPath(); + // 处理分区 + findCurrentPartition(path); + } + OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(hadoopJobConf); + readerOptions.filesystem(fs); + + org.apache.hadoop.hive.ql.io.orc.Reader reader; + try { + reader = OrcFile.createReader(path, readerOptions); + } catch (IOException e) { + LOG.error( + "Create reader error, path = {}, Exception = {}", + path, + ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException("Create reader error."); + } + + String typeStruct = reader.getObjectInspector().getTypeName(); + if (StringUtils.isEmpty(typeStruct)) { + throw new RuntimeException("can't retrieve type struct from " + path); + } + + int startIndex = typeStruct.indexOf("<") + 1; + int endIndex = typeStruct.lastIndexOf(">"); + typeStruct = typeStruct.substring(startIndex, endIndex); + startIndex = typeStruct.indexOf("<") + 1; + endIndex = typeStruct.lastIndexOf(">"); + if (startIndex == 0 || endIndex == -1) { + LOG.error( + "Please check whether the transaction table option(hiveTransactionTable=true) is used to synchronize non-transactional tables(check hdfs file format,except for base or delta), typeStruct = {}", + typeStruct); + throw new RuntimeException( + "Please check whether the transaction table option(hiveTransactionTable=true) is used to synchronize non-transactional tables.(check hdfs file format,except for base or delta)"); + } + // 如果读非事务表的 hdfs 文件,此处会解析类型报错索引异常。 + typeStruct = typeStruct.substring(startIndex, endIndex); + + if (typeStruct.matches(COMPLEX_FIELD_TYPE_SYMBOL_REGEX)) { + throw new RuntimeException( + "Field types such as array, map, and struct are not supported."); + } + + List cols = parseColumnAndType(typeStruct); + + fullColNames = new String[cols.size()]; + String[] fullColTypes = new String[cols.size()]; + + for (int i = 0; i < cols.size(); ++i) { + String[] nameTypeTuple2 = cols.get(i).split(":"); + fullColNames[i] = nameTypeTuple2[0]; + fullColTypes[i] = nameTypeTuple2[1]; + } + final String names = StringUtils.join(fullColNames, ","); + final String types = StringUtils.join(fullColTypes, ":"); + Properties p = new Properties(); + p.setProperty("columns", names); + p.setProperty("columns.types", types); + + OrcSerde orcSerde = new OrcSerde(); + orcSerde.initialize(hadoopJobConf, p); + + try { + this.inspector = (StructObjectInspector) orcSerde.getObjectInspector(); + } catch (SerDeException e) { + throw new RuntimeException("hive transaction table inspector create failed."); + } + // 读 hive 事务表需要设置的属性 + hadoopJobConf.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, names); + // int:bigint:string:float:double:struct + hadoopJobConf.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, types); + AcidUtils.setAcidOperationalProperties(hadoopJobConf, true, null); + try { + recordReader = inputFormat.getRecordReader(orcSplit, hadoopJobConf, Reporter.NULL); + } catch (IOException e) { + throw new RuntimeException("hive transaction table record reader creation failed."); + } + key = recordReader.createKey(); + value = recordReader.createValue(); + fields = inspector.getAllStructFieldRefs(); + } + + @Override + public InputFormat createMapredInputFormat() { + return new OrcInputFormat(); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3InputFormatBuilder.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3InputFormatBuilder.java new file mode 100644 index 0000000000..ca34275021 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3InputFormatBuilder.java @@ -0,0 +1,59 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.source; + +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.enums.FileType; +import com.dtstack.chunjun.source.format.BaseRichInputFormatBuilder; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; + +/** @author liuliu 2022/3/23 */ +public class Hive3InputFormatBuilder extends BaseRichInputFormatBuilder { + BaseHdfsInputFormat inputFormat; + + public Hive3InputFormatBuilder(String fileType, boolean isHiveTransactionTable) { + super(null); + switch (FileType.getByName(fileType)) { + case ORC: + if (isHiveTransactionTable) { + inputFormat = new HdfsTransactionInputFormat(); + } else { + inputFormat = new HdfsOrcInputFormat(); + } + break; + case PARQUET: + inputFormat = new HdfsParquetInputFormat(); + break; + case TEXT: + inputFormat = new HdfsTextInputFormat(); + break; + default: + throw new UnsupportedTypeException(fileType); + } + super.format = this.inputFormat; + } + + @Override + protected void checkFormat() {} + + public void setHdfsConf(HdfsConf hdfsConf) { + super.setConfig(hdfsConf); + inputFormat.sethdfsConf(hdfsConf); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3SourceFactory.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3SourceFactory.java new file mode 100644 index 0000000000..0863c93c2d --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3SourceFactory.java @@ -0,0 +1,69 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.source; + +import com.dtstack.chunjun.conf.SyncConf; +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.converter.HdfsRawTypeConverter; +import com.dtstack.chunjun.connector.hive3.util.Hive3Util; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.RawTypeConverter; +import com.dtstack.chunjun.source.SourceFactory; +import com.dtstack.chunjun.util.GsonUtil; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; + +/** @author liuliu 2022/3/23 */ +public class Hive3SourceFactory extends SourceFactory { + HdfsConf hdfsConf; + + public Hive3SourceFactory(SyncConf syncConf, StreamExecutionEnvironment env) { + super(syncConf, env); + hdfsConf = + GsonUtil.GSON.fromJson( + GsonUtil.GSON.toJson(syncConf.getReader().getParameter()), HdfsConf.class); + hdfsConf.setColumn(syncConf.getReader().getFieldList()); + super.initCommonConf(hdfsConf); + } + + @Override + public RawTypeConverter getRawTypeConverter() { + return HdfsRawTypeConverter::apply; + } + + @Override + public DataStream createSource() { + Hive3InputFormatBuilder builder = + new Hive3InputFormatBuilder( + hdfsConf.getFileType(), hdfsConf.isHiveTransactionTable()); + builder.setHdfsConf(hdfsConf); + AbstractRowConverter rowConverter = + Hive3Util.createRowConverter( + useAbstractBaseColumn, + hdfsConf.getFileType(), + hdfsConf.getColumn(), + getRawTypeConverter(), + hdfsConf); + + builder.setRowConverter(rowConverter, useAbstractBaseColumn); + return createInput(builder.finish()); + } +} diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/util/Hive3Util.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/util/Hive3Util.java new file mode 100644 index 0000000000..cdba40eaa2 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/util/Hive3Util.java @@ -0,0 +1,601 @@ +/* + * 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 com.dtstack.chunjun.connector.hive3.util; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hive3.conf.HdfsConf; +import com.dtstack.chunjun.connector.hive3.converter.HdfsOrcColumnConverter; +import com.dtstack.chunjun.connector.hive3.converter.HdfsOrcRowConverter; +import com.dtstack.chunjun.connector.hive3.converter.HdfsParquetColumnConverter; +import com.dtstack.chunjun.connector.hive3.converter.HdfsParquetRowConverter; +import com.dtstack.chunjun.connector.hive3.converter.HdfsTextColumnConverter; +import com.dtstack.chunjun.connector.hive3.converter.HdfsTextRowConverter; +import com.dtstack.chunjun.connector.hive3.enums.FileType; +import com.dtstack.chunjun.connector.hive3.source.HdfsPathFilter; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.RawTypeConverter; +import com.dtstack.chunjun.enums.ColumnType; +import com.dtstack.chunjun.security.KerberosUtil; +import com.dtstack.chunjun.throwable.UnsupportedTypeException; +import com.dtstack.chunjun.util.ColumnTypeUtil; +import com.dtstack.chunjun.util.ExceptionUtil; +import com.dtstack.chunjun.util.TableUtil; + +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.Date; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableHiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.ByteWritable; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.parquet.io.api.Binary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.security.PrivilegedAction; +import java.sql.Timestamp; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.dtstack.chunjun.security.KerberosUtil.KRB_STR; + +/** @author liuliu 2022/3/22 */ +public class Hive3Util { + private static final Logger LOG = LoggerFactory.getLogger(Hive3Util.class); + public static final String NULL_VALUE = "\\N"; + + private static final long NANO_SECONDS_PER_DAY = 86400_000_000_000L; + + private static final long JULIAN_EPOCH_OFFSET_DAYS = 2440588; + + private static final double SCALE_TWO = 2.0; + private static final double SCALE_TEN = 10.0; + private static final int BIT_SIZE = 8; + + private static final String KEY_HADOOP_SECURITY_AUTHORIZATION = "hadoop.security.authorization"; + private static final String KEY_HADOOP_SECURITY_AUTHENTICATION = + "hadoop.security.authentication"; + private static final String KEY_DEFAULT_FS = "fs.default.name"; + private static final String KEY_FS_HDFS_IMPL_DISABLE_CACHE = "fs.hdfs.impl.disable.cache"; + private static final String KEY_HA_DEFAULT_FS = "fs.defaultFS"; + private static final String KEY_DFS_NAME_SERVICES = "dfs.nameservices"; + private static final String KEY_HADOOP_USER_NAME = "hadoop.user.name"; + + public static Object getWritableValue(Object writable) { + Class clz = writable.getClass(); + Object ret; + + if (clz == IntWritable.class) { + ret = ((IntWritable) writable).get(); + } else if (clz == Text.class) { + ret = writable.toString(); + } else if (clz == LongWritable.class) { + ret = ((LongWritable) writable).get(); + } else if (clz == ByteWritable.class) { + ret = ((ByteWritable) writable).get(); + } else if (clz == DateWritable.class) { + ret = ((DateWritable) writable).get(); + } else if (writable instanceof DoubleWritable) { + ret = ((DoubleWritable) writable).get(); + } else if (writable instanceof TimestampWritable) { + ret = ((TimestampWritable) writable).getTimestamp(); + } else if (writable instanceof TimestampWritableV2) { + ret = ((TimestampWritableV2) writable).getTimestamp(); + } else if (writable instanceof HiveDecimalWritable) { + ret = ((HiveDecimalWritable) writable).getHiveDecimal().bigDecimalValue(); + } else if (writable instanceof DateWritable) { + ret = ((DateWritable) writable).get(); + } else if (writable instanceof DateWritableV2) { + ret = ((DateWritableV2) writable).get(); + } else if (writable instanceof FloatWritable) { + ret = ((FloatWritable) writable).get(); + } else if (writable instanceof BooleanWritable) { + ret = ((BooleanWritable) writable).get(); + } else if (writable instanceof BytesWritable) { + BytesWritable bytesWritable = (BytesWritable) writable; + byte[] bytes = bytesWritable.getBytes(); + // org.apache.hadoop.io.BytesWritable.setSize方法中扩容导致byte[]末尾自动补0,这里需要把末尾的0去掉才能得到真正的byte[] + ret = new byte[bytesWritable.getLength()]; + System.arraycopy(bytes, 0, ret, 0, bytesWritable.getLength()); + } else if (writable instanceof HiveDecimalWritable) { + ret = ((HiveDecimalWritable) writable).getHiveDecimal().bigDecimalValue(); + } else if (writable instanceof ShortWritable) { + ret = ((ShortWritable) writable).get(); + } else { + ret = writable.toString(); + } + return ret; + } + + public static AbstractRowConverter createRowConverter( + boolean useAbstractBaseColumn, + String fileType, + List fieldConfList, + RawTypeConverter rawTypeConverter, + HdfsConf hdfsConf) { + AbstractRowConverter rowConverter; + RowType rowType = TableUtil.createRowType(fieldConfList, rawTypeConverter); + if (useAbstractBaseColumn) { + switch (FileType.getByName(fileType)) { + case ORC: + rowConverter = new HdfsOrcColumnConverter(rowType, hdfsConf); + break; + case PARQUET: + rowConverter = new HdfsParquetColumnConverter(rowType, hdfsConf); + break; + case TEXT: + rowConverter = new HdfsTextColumnConverter(rowType, hdfsConf); + break; + default: + throw new UnsupportedTypeException(fileType); + } + } else { + switch (FileType.getByName(fileType)) { + case ORC: + rowConverter = new HdfsOrcRowConverter(rowType, hdfsConf); + break; + case PARQUET: + rowConverter = new HdfsParquetRowConverter(rowType, hdfsConf); + break; + case TEXT: + rowConverter = new HdfsTextRowConverter(rowType, hdfsConf); + break; + default: + throw new UnsupportedTypeException(fileType); + } + } + return rowConverter; + } + + /** + * Encapsulate common exceptions in hdfs operation and give solutions + * + * @param customizeMessage + * @param errorMsg + * @return + */ + public static String parseErrorMsg(String customizeMessage, String errorMsg) { + StringBuilder str = new StringBuilder(); + str.append(customizeMessage); + Pair pair = null; + if (StringUtils.isNotBlank(customizeMessage)) { + str.append(customizeMessage); + } + if (StringUtils.isNotBlank(errorMsg)) { + if (errorMsg.contains( + "at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease")) { + pair = + Pair.of( + "The file or directory may not exist or may be inaccessible ", + "make sure there is no other task operating same hdfs dir at same time"); + } + } + if (pair != null) { + str.append("\nthe Cause maybe : ") + .append(pair.getLeft()) + .append(", \nand the Solution maybe : ") + .append(pair.getRight()) + .append(", "); + } + + return str.toString(); + } + + public static Binary timeToBinary(Timestamp timestamp) { + long nano = timestamp.getTime() * 1000_000; + + int julianDays = (int) ((nano / NANO_SECONDS_PER_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + byte[] julianDaysBytes = getBytes(julianDays); + flip(julianDaysBytes); + + long lastDayNanos = nano % NANO_SECONDS_PER_DAY; + byte[] lastDayNanosBytes = getBytes(lastDayNanos); + flip(lastDayNanosBytes); + + byte[] dst = new byte[12]; + + System.arraycopy(lastDayNanosBytes, 0, dst, 0, 8); + System.arraycopy(julianDaysBytes, 0, dst, 8, 4); + + return Binary.fromConstantByteArray(dst); + } + + private static byte[] getBytes(long i) { + byte[] bytes = new byte[8]; + bytes[0] = (byte) ((i >> 56) & 0xFF); + bytes[1] = (byte) ((i >> 48) & 0xFF); + bytes[2] = (byte) ((i >> 40) & 0xFF); + bytes[3] = (byte) ((i >> 32) & 0xFF); + bytes[4] = (byte) ((i >> 24) & 0xFF); + bytes[5] = (byte) ((i >> 16) & 0xFF); + bytes[6] = (byte) ((i >> 8) & 0xFF); + bytes[7] = (byte) (i & 0xFF); + return bytes; + } + + /** @param bytes */ + private static void flip(byte[] bytes) { + for (int i = 0, j = bytes.length - 1; i < j; i++, j--) { + byte t = bytes[i]; + bytes[i] = bytes[j]; + bytes[j] = t; + } + } + + public static Binary decimalToBinary(final HiveDecimal hiveDecimal, int prec, int scale) { + byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray(); + + // Estimated number of bytes needed. + int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1]; + if (precToBytes == decimalBytes.length) { + // No padding needed. + return Binary.fromReusedByteArray(decimalBytes); + } + + byte[] tgt = new byte[precToBytes]; + if (hiveDecimal.signum() == -1) { + // For negative number, initializing bits to 1 + for (int i = 0; i < precToBytes; i++) { + tgt[i] |= 0xFF; + } + } + + // Padding leading zeroes/ones. + System.arraycopy( + decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); + return Binary.fromReusedByteArray(tgt); + } + + public static int computeMinBytesForPrecision(int precision) { + int numBytes = 1; + while (Math.pow(SCALE_TWO, BIT_SIZE * numBytes - 1.0) < Math.pow(SCALE_TEN, precision)) { + numBytes += 1; + } + return numBytes; + } + + public static ObjectInspector columnTypeToObjectInspector( + ColumnType columnType, ColumnTypeUtil.DecimalInfo decimalInfo) { + ObjectInspector objectInspector; + switch (columnType) { + case TINYINT: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Byte.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case SMALLINT: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Short.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case INT: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BIGINT: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case FLOAT: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Float.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DOUBLE: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Double.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DECIMAL: + if (decimalInfo != null) { + try { + Constructor constructor = + WritableHiveDecimalObjectInspector.class.getDeclaredConstructor( + DecimalTypeInfo.class); + constructor.setAccessible(true); + objectInspector = + constructor.newInstance( + new DecimalTypeInfo( + decimalInfo.getPrecision(), + decimalInfo.getScale())); + } catch (Exception e) { + LOG.warn( + "can't create WritableHiveDecimalObjectInspector from {}", + decimalInfo, + e); + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + HiveDecimalWritable.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + } + } else { + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + HiveDecimalWritable.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + } + break; + case TIMESTAMP: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + org.apache.hadoop.hive.common.type.Timestamp.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DATE: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Date.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case STRING: + case VARCHAR: + case CHAR: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + String.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BOOLEAN: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Boolean.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BINARY: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + BytesWritable.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + default: + throw new IllegalArgumentException("You should not be here"); + } + return objectInspector; + } + + public static Set getAllPartitionPath( + String inputPath, FileSystem fileSystem, HdfsPathFilter pathFilter) { + // 因为拿到的是所有文件,分区可能重复。此处用 hashSet 避免分区路径重复。 + HashSet paths = new HashSet<>(); + try { + findAllPartitionPath(paths, fileSystem, inputPath, pathFilter); + } catch (IOException e) { + LOG.error( + "retrieve all partition error, hdfs input path {},errMsg {}", + inputPath, + ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException("retrieve ClassLoad happens error"); + } + return paths; + } + + /** + * 递归 hdfs 分区目录 + * + * @param paths 所有的分区目录 + * @param fileSystem 文件系统 + * @param path 传入的 path + * @throws IOException IOException + */ + private static void findAllPartitionPath( + HashSet paths, FileSystem fileSystem, String path, HdfsPathFilter pathFilter) + throws IOException { + FileStatus[] allChildPath = fileSystem.listStatus(new Path(path), pathFilter); + if (allChildPath != null && allChildPath.length > 0) { + for (FileStatus filePath : allChildPath) { + if (filePath.isDirectory() + && getFileName(filePath.getPath().toString()).contains("=")) { + findAllPartitionPath( + paths, fileSystem, filePath.getPath().toString(), pathFilter); + } else { + paths.add(filePath.getPath().getParent().toString()); + return; + } + } + } + } + + public static JobConf getJobConf(Map confMap, String defaultFs) { + confMap = fillConfig(confMap, defaultFs); + + JobConf jobConf = new JobConf(); + confMap.forEach( + (key, val) -> { + if (val != null) { + jobConf.set(key, val.toString()); + } + }); + + return jobConf; + } + + private static String getFileName(String path) { + return path.substring(path.lastIndexOf("/")); + } + + public static Configuration getConfiguration(Map confMap, String defaultFs) { + confMap = fillConfig(confMap, defaultFs); + + Configuration conf = new Configuration(); + confMap.forEach( + (key, val) -> { + if (val != null) { + conf.set(key, val.toString()); + } + }); + + return conf; + } + + public static boolean isOpenKerberos(Map hadoopConfig) { + if (!MapUtils.getBoolean(hadoopConfig, KEY_HADOOP_SECURITY_AUTHORIZATION, false)) { + return false; + } + + return KRB_STR.equalsIgnoreCase( + MapUtils.getString(hadoopConfig, KEY_HADOOP_SECURITY_AUTHENTICATION)); + } + + public static void setHadoopUserName(Configuration conf) { + String hadoopUserName = conf.get(KEY_HADOOP_USER_NAME); + if (org.apache.commons.lang.StringUtils.isEmpty(hadoopUserName)) { + return; + } + + try { + String previousUserName = UserGroupInformation.getLoginUser().getUserName(); + LOG.info( + "Hadoop user from '{}' switch to '{}' with SIMPLE auth", + previousUserName, + hadoopUserName); + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(hadoopUserName); + UserGroupInformation.setLoginUser(ugi); + } catch (Exception e) { + LOG.warn("Set hadoop user name error:", e); + } + } + + private static FileSystem getFsWithUser( + Map hadoopConfig, String defaultFs, String user) throws Exception { + if (org.apache.commons.lang.StringUtils.isEmpty(user)) { + return FileSystem.get(getConfiguration(hadoopConfig, defaultFs)); + } + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user); + return ugi.doAs( + (PrivilegedAction) + () -> { + try { + return FileSystem.get(getConfiguration(hadoopConfig, defaultFs)); + } catch (Exception e) { + throw new RuntimeException("Get FileSystem error:", e); + } + }); + } + + public static FileSystem getFileSystem( + Map hadoopConfigMap, + String defaultFs, + String user, + DistributedCache distributedCache) + throws Exception { + if (isOpenKerberos(hadoopConfigMap)) { + return getFsWithKerberos(hadoopConfigMap, defaultFs, distributedCache); + } + return getFsWithUser(hadoopConfigMap, defaultFs, user); + } + + public static FileSystem getFileSystem( + Map hadoopConfigMap, + String defaultFs, + DistributedCache distributedCache) + throws Exception { + if (isOpenKerberos(hadoopConfigMap)) { + return getFsWithKerberos(hadoopConfigMap, defaultFs, distributedCache); + } + + Configuration conf = getConfiguration(hadoopConfigMap, defaultFs); + setHadoopUserName(conf); + + return FileSystem.get(getConfiguration(hadoopConfigMap, defaultFs)); + } + + private static FileSystem getFsWithKerberos( + Map hadoopConfig, String defaultFs, DistributedCache distributedCache) + throws Exception { + UserGroupInformation ugi = getUGI(hadoopConfig, defaultFs, distributedCache); + + return ugi.doAs( + (PrivilegedAction) + () -> { + try { + return FileSystem.get(getConfiguration(hadoopConfig, defaultFs)); + } catch (Exception e) { + throw new RuntimeException( + "Get FileSystem with kerberos error:", e); + } + }); + } + + public static UserGroupInformation getUGI( + Map hadoopConfig, String defaultFs, DistributedCache distributedCache) + throws IOException { + String keytabFileName = KerberosUtil.getPrincipalFileName(hadoopConfig); + keytabFileName = KerberosUtil.loadFile(hadoopConfig, keytabFileName, distributedCache); + String principal = KerberosUtil.getPrincipal(hadoopConfig, keytabFileName); + KerberosUtil.loadKrb5Conf(hadoopConfig, distributedCache); + KerberosUtil.refreshConfig(); + + return KerberosUtil.loginAndReturnUgi( + getConfiguration(hadoopConfig, defaultFs), principal, keytabFileName); + } + + private static Map fillConfig(Map confMap, String defaultFs) { + if (confMap == null) { + confMap = new HashMap<>(); + } + + if (isHaMode(confMap)) { + if (defaultFs != null) { + confMap.put(KEY_HA_DEFAULT_FS, defaultFs); + } + } else { + if (defaultFs != null) { + confMap.put(KEY_DEFAULT_FS, defaultFs); + } + } + + confMap.put(KEY_FS_HDFS_IMPL_DISABLE_CACHE, "true"); + return confMap; + } + + private static boolean isHaMode(Map confMap) { + return org.apache.commons.lang.StringUtils.isNotEmpty( + MapUtils.getString(confMap, KEY_DFS_NAME_SERVICES)); + } +} diff --git a/chunjun-connectors/pom.xml b/chunjun-connectors/pom.xml index 51119b336c..46504affc1 100644 --- a/chunjun-connectors/pom.xml +++ b/chunjun-connectors/pom.xml @@ -51,6 +51,7 @@ chunjun-connector-ftp + diff --git a/pom.xml b/pom.xml index 41dc0d1d7f..0219373965 100644 --- a/pom.xml +++ b/pom.xml @@ -14,15 +14,15 @@ chunjun-core chunjun-connectors chunjun-clients - + chunjun-formats chunjun-docker chunjun-metrics chunjun-dirty chunjun-restore - chunjun-sql - chunjun-assembly - + chunjun-sql + chunjun-assembly + UTF-8 @@ -415,5 +415,19 @@ + + + + hive3 + + 3.1.4 + + + + chunjun-connectors/chunjun-connector-hive3 + + + + From e5aba29489468077bbb08abb5a09d8843c03ee33 Mon Sep 17 00:00:00 2001 From: wujuan Date: Sun, 3 Jul 2022 19:17:44 +0800 Subject: [PATCH 2/3] [feat][hive3] The class conflict of hadoop3 and hive3 be resolved, and add hive3 transaction table json. --- bin/submit.sh | 8 +- chunjun-clients/pom.xml | 433 +++++++------ .../chunjun-connector-hbase-base/pom.xml | 1 - .../chunjun-connector-hive3/pom.xml | 365 +++++++++-- .../connector/hive3/conf/HdfsConf.java | 15 +- .../hive3/sink/HdfsOrcOutputFormat.java | 2 +- .../hive3/source/Hive3SourceFactory.java | 3 +- .../provider/DruidDataSourceProvider.java | 3 +- .../chunjun-connector-stream/pom.xml | 33 +- chunjun-connectors/pom.xml | 59 +- chunjun-core/pom.xml | 567 +++++++++--------- .../json/hive3/hive3_orc_stream.json | 57 ++ .../json/hive3/hive3_transaction_stream.json | 58 ++ .../json/hive3/stream_hive3_orc.json | 79 +++ .../json/hive3/stream_hive3_transaction.json | 89 +++ .../chunjun-restore-common/pom.xml | 7 +- pom.xml | 84 +-- 17 files changed, 1283 insertions(+), 580 deletions(-) create mode 100644 chunjun-examples/json/hive3/hive3_orc_stream.json create mode 100644 chunjun-examples/json/hive3/hive3_transaction_stream.json create mode 100644 chunjun-examples/json/hive3/stream_hive3_orc.json create mode 100644 chunjun-examples/json/hive3/stream_hive3_transaction.json diff --git a/bin/submit.sh b/bin/submit.sh index e85cb72817..ed80a8bd23 100644 --- a/bin/submit.sh +++ b/bin/submit.sh @@ -31,8 +31,12 @@ else fi fi -export CHUNJUN_HOME="$(cd "`dirname "$0"`"/..; pwd)" -JAR_DIR=$CHUNJUN_HOME/lib/* +if [[ $CHUNJUN_HOME && -z $CHUNJUN_HOME ]];then + export CHUNJUN_HOME=$CHUNJUN_HOME +else + export CHUNJUN_HOME="$(cd "`dirname "$0"`"/../chunjun-dist; pwd)" +fi +JAR_DIR=$CHUNJUN_HOME/../lib/* CLASS_NAME=com.dtstack.chunjun.client.Launcher JOBTYPE="sync" diff --git a/chunjun-clients/pom.xml b/chunjun-clients/pom.xml index 3ea28a0135..5c5fb0da4b 100644 --- a/chunjun-clients/pom.xml +++ b/chunjun-clients/pom.xml @@ -1,104 +1,201 @@ - - chunjun - com.dtstack.chunjun - 1.12-SNAPSHOT - - 4.0.0 - - chunjun-clients - ChunJun : Clients + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + chunjun + com.dtstack.chunjun + 1.12-SNAPSHOT + + 4.0.0 - + chunjun-clients + ChunJun : Clients + org.apache.flink flink-kubernetes_${scala.binary.version} ${flink.version} + + + slf4j-api + org.slf4j + + + + + + com.dtstack.chunjun + chunjun-core + ${project.version} + + + commons-logging + commons-logging + + + logback-classic + ch.qos.logback + + + logback-core + ch.qos.logback + + + minlog + com.esotericsoftware.minlog + + - - com.dtstack.chunjun - chunjun-core - ${project.version} - - - - commons-codec - commons-codec - 1.9 - - - - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - - - commons-cli - commons-cli - + + commons-codec + commons-codec + 1.9 + + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + + + commons-cli + commons-cli + netty-all io.netty - - - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - - - commons-cli - commons-cli - - - - - - org.apache.hadoop - hadoop-yarn-common - ${hadoop.version} - - - commons-cli - commons-cli - - - - - - org.apache.hadoop - hadoop-yarn-client - ${hadoop.version} - - - commons-cli - commons-cli - - - - - org.apache.hadoop - hadoop-mapreduce-client-core - ${hadoop.version} - - - org.apache.hadoop - hadoop-yarn-api - ${hadoop.version} - - - org.apache.avro - avro - 1.8.2 - + + log4j + log4j + + + commons-logging + commons-logging + + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + commons-cli + commons-cli + + + guava + com.google.guava + + + slf4j-api + org.slf4j + + + log4j + log4j + + + slf4j-log4j12 + org.slf4j + + + commons-logging + commons-logging + + + + + + org.apache.hadoop + hadoop-yarn-common + ${hadoop.version} + + + commons-cli + commons-cli + + + guava + com.google.guava + + + slf4j-api + org.slf4j + + + log4j + log4j + + + commons-logging + commons-logging + + + + + + org.apache.hadoop + hadoop-yarn-client + ${hadoop.version} + + + commons-cli + commons-cli + + + log4j + log4j + + + commons-logging + commons-logging + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + + + slf4j-api + org.slf4j + + + slf4j-log4j12 + org.slf4j + + + + + org.apache.hadoop + hadoop-yarn-api + ${hadoop.version} + + + commons-logging + commons-logging + + + + + org.apache.avro + avro + 1.8.2 + + + slf4j-api + org.slf4j + + + org.apache.flink @@ -106,80 +203,80 @@ ${flink.version} - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - package - - shade - - - false - - - - - - - - reference.conf - - - - core-default.xml - - - - core-site.xml - - - - yarn-default.xml - - - - mapred-default.xml - - - - mapred-site.xml - - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - - - - maven-antrun-plugin - - - copy-resources - - package - - run - - - + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + false + + + + + + + + reference.conf + + + + core-default.xml + + + + core-site.xml + + + + yarn-default.xml + + + + mapred-default.xml + + + + mapred-site.xml + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + + + copy-resources + + package + + run + + + - - - - - - - + + + + + + + diff --git a/chunjun-connectors/chunjun-connector-hbase-base/pom.xml b/chunjun-connectors/chunjun-connector-hbase-base/pom.xml index 5de6fa52ed..bf17fef772 100644 --- a/chunjun-connectors/chunjun-connector-hbase-base/pom.xml +++ b/chunjun-connectors/chunjun-connector-hbase-base/pom.xml @@ -33,7 +33,6 @@ 1.4.3 - 2.4.1 2.12.1 diff --git a/chunjun-connectors/chunjun-connector-hive3/pom.xml b/chunjun-connectors/chunjun-connector-hive3/pom.xml index 00debe640b..913587d1f5 100644 --- a/chunjun-connectors/chunjun-connector-hive3/pom.xml +++ b/chunjun-connectors/chunjun-connector-hive3/pom.xml @@ -15,12 +15,30 @@ 8 8 - 3.1.3 + 3.1.2 3.1.4 - + + + org.slf4j + slf4j-api + ${slf4j.version} + provided + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + provided + + + log4j + log4j + ${log4j.version} + provided + org.apache.hadoop @@ -39,9 +57,16 @@ commons-cli commons-cli + + commons-logging + commons-logging + + + log4j + log4j + - org.apache.hadoop hadoop-common @@ -55,38 +80,192 @@ com.google.protobuf protobuf-java-util + + guava + com.google.guava + + + avro + org.apache.avro + + + commons-cli + commons-cli + + + zookeeper + org.apache.zookeeper + + + slf4j-api + org.slf4j + + + commons-logging + commons-logging + + + log4j + log4j + + + slf4j-log4j12 + org.slf4j + - org.apache.hadoop hadoop-yarn-common ${hadoop3.version} + + + guava + com.google.guava + + + guice + com.google.inject + + + commons-cli + commons-cli + + + commons-logging + commons-logging + + + log4j + log4j + + - org.apache.hadoop hadoop-yarn-client ${hadoop3.version} + + + commons-cli + commons-cli + + + commons-logging + commons-logging + + + log4j + log4j + + org.apache.hadoop hadoop-mapreduce-client-core ${hadoop3.version} + + + avro + org.apache.avro + + + slf4j-log4j12 + org.slf4j + + org.apache.hadoop hadoop-yarn-api ${hadoop3.version} + + + commons-logging + commons-logging + + - org.apache.hive hive-exec ${hive.version} + + + guava + com.google.guava + + + hadoop-annotations + org.apache.hadoop + + + hadoop-auth + org.apache.hadoop + + + hadoop-common + org.apache.hadoop + + + hadoop-yarn-api + org.apache.hadoop + + + hadoop-yarn-common + org.apache.hadoop + + + antlr-runtime + org.antlr + + + calcite-core + org.apache.calcite + + + commons-cli + commons-cli + + + zookeeper + org.apache.zookeeper + + + slf4j-api + org.slf4j + + + jetty-util-ajax + org.eclipse.jetty + + + commons-logging + commons-logging + + + log4j + log4j + + + log4j-1.2-api + org.apache.logging.log4j + + + log4j-api + org.apache.logging.log4j + + + log4j-core + org.apache.logging.log4j + + + log4j-slf4j-impl + org.apache.logging.log4j + + - org.apache.hive hive-serde @@ -100,13 +279,38 @@ org.xerial.snappy snappy-java + + guava + com.google.guava + + + httpclient + org.apache.httpcomponents + + + avro + org.apache.avro + + + commons-cli + commons-cli + + + slf4j-api + org.slf4j + - org.apache.hive hive-streaming ${hive.version} + + + slf4j-api + org.slf4j + + @@ -128,27 +332,33 @@ org.xerial.snappy snappy-java + + slf4j-api + org.slf4j + - org.xerial.snappy snappy-java 1.1.4 - httpcore org.apache.httpcomponents 4.4.5 - httpclient + + + commons-logging + commons-logging + + org.apache.httpcomponents 4.5.2 - org.apache.hive hive-metastore @@ -162,58 +372,135 @@ org.datanucleus datanucleus-rdbms + + guava + com.google.guava + + + hadoop-auth + org.apache.hadoop + + + hadoop-common + org.apache.hadoop + + + guice + com.google.inject + + + zookeeper + org.apache.zookeeper + + + slf4j-api + org.slf4j + + + orc-core + org.apache.orc + + + commons-logging + commons-logging + + + log4j + log4j + + + log4j-1.2-api + org.apache.logging.log4j + + + log4j-slf4j-impl + org.apache.logging.log4j + - org.apache.hive hive-common ${hive.version} + + + hadoop-common + org.apache.hadoop + + + hadoop-hdfs + org.apache.hadoop + + + commons-cli + commons-cli + + + slf4j-api + org.slf4j + + + jetty-webapp + org.eclipse.jetty + + + commons-logging + commons-logging + + + log4j-1.2-api + org.apache.logging.log4j + + + log4j-api + org.apache.logging.log4j + + + log4j-core + org.apache.logging.log4j + + + log4j-slf4j-impl + org.apache.logging.log4j + + - org.datanucleus datanucleus-core 5.0.1 - org.datanucleus datanucleus-api-jdo 5.0.1 - org.datanucleus datanucleus-rdbms 5.0.1 - org.apache.commons commons-collections4 4.3 - - - com.google.guava - guava - 27.0-jre - - - - com.dtstack.chunjun - chunjun-connector-stream - 1.12-SNAPSHOT - - commons-cli commons-cli 1.3.1 provided - + + + + + + + org.apache.avro + avro + 1.8.2 + @@ -241,14 +528,18 @@ + + + + - com.google.common - shade.hive3.com.google.common - - - com.google.guava - shade.hive3.com.google.guava + com.google.common.base + shade.hive3.com.google.common.base + + + + org.checkerframework shade.hive3.org.checkerframework diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/conf/HdfsConf.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/conf/HdfsConf.java index 1e6bd7dba1..9db89d0c47 100644 --- a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/conf/HdfsConf.java +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/conf/HdfsConf.java @@ -43,8 +43,6 @@ public class HdfsConf extends BaseFileConf { private int[] fullColumnIndexes; /** hive3 事务表 */ - private boolean hiveTransactionTable = false; - private boolean isTransaction = false; private String partitionName; @@ -131,14 +129,6 @@ public void setFullColumnIndexes(int[] fullColumnIndexes) { this.fullColumnIndexes = fullColumnIndexes; } - public boolean isHiveTransactionTable() { - return hiveTransactionTable; - } - - public void setHiveTransactionTable(boolean hiveTransactionTable) { - this.hiveTransactionTable = hiveTransactionTable; - } - public String getPartitionName() { return partitionName; } @@ -198,8 +188,6 @@ public String toString() { + fullColumnType + ", fullColumnIndexes=" + Arrays.toString(fullColumnIndexes) - + ", hiveTransactionTable=" - + hiveTransactionTable + ", isTransaction=" + isTransaction + ", partitionName='" @@ -211,7 +199,6 @@ public String toString() { + ", table='" + table + '\'' - + '}' - + super.toString(); + + '}'; } } diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsOrcOutputFormat.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsOrcOutputFormat.java index 9a688b25a8..5639fbbf66 100644 --- a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsOrcOutputFormat.java +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/sink/HdfsOrcOutputFormat.java @@ -72,7 +72,7 @@ public class HdfsOrcOutputFormat extends BaseHdfsOutputFormat { // kerberos 认证 protected transient UserGroupInformation ugi; protected boolean openKerberos; - public static final String HADOOP_USER_NAME = "HADOOP_USER_NAME"; + public static final String HADOOP_USER_NAME = "hadoop.user.name"; public static final String KEY_PRINCIPAL = "principal"; protected final Logger LOG = LoggerFactory.getLogger(HdfsOrcOutputFormat.class); diff --git a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3SourceFactory.java b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3SourceFactory.java index 0863c93c2d..88ae2d9a28 100644 --- a/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3SourceFactory.java +++ b/chunjun-connectors/chunjun-connector-hive3/src/main/java/com/dtstack/chunjun/connector/hive3/source/Hive3SourceFactory.java @@ -52,8 +52,7 @@ public RawTypeConverter getRawTypeConverter() { @Override public DataStream createSource() { Hive3InputFormatBuilder builder = - new Hive3InputFormatBuilder( - hdfsConf.getFileType(), hdfsConf.isHiveTransactionTable()); + new Hive3InputFormatBuilder(hdfsConf.getFileType(), hdfsConf.isTransaction()); builder.setHdfsConf(hdfsConf); AbstractRowConverter rowConverter = Hive3Util.createRowConverter( diff --git a/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/lookup/provider/DruidDataSourceProvider.java b/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/lookup/provider/DruidDataSourceProvider.java index ae8477db47..ce05d6c676 100644 --- a/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/lookup/provider/DruidDataSourceProvider.java +++ b/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/lookup/provider/DruidDataSourceProvider.java @@ -18,9 +18,8 @@ package com.dtstack.chunjun.connector.jdbc.lookup.provider; -import org.apache.flink.shaded.guava18.com.google.common.base.CaseFormat; - import com.alibaba.druid.pool.DruidDataSource; +import com.google.common.base.CaseFormat; import io.vertx.core.json.JsonObject; import io.vertx.ext.jdbc.spi.DataSourceProvider; diff --git a/chunjun-connectors/chunjun-connector-stream/pom.xml b/chunjun-connectors/chunjun-connector-stream/pom.xml index c053bb9cda..bb1274b0dd 100644 --- a/chunjun-connectors/chunjun-connector-stream/pom.xml +++ b/chunjun-connectors/chunjun-connector-stream/pom.xml @@ -12,13 +12,32 @@ chunjun-connector-stream ChunJun : Connectors : Stream - - - com.github.jsonzou - jmockdata - 4.2.0 - - + + + + org.slf4j + slf4j-api + ${slf4j.version} + provided + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + provided + + + log4j + log4j + ${log4j.version} + provided + + + com.github.jsonzou + jmockdata + 4.2.0 + + diff --git a/chunjun-connectors/pom.xml b/chunjun-connectors/pom.xml index 46504affc1..ad28870f38 100644 --- a/chunjun-connectors/pom.xml +++ b/chunjun-connectors/pom.xml @@ -46,19 +46,15 @@ chunjun-connector-cassandra - chunjun-connector-file + chunjun-connector-file chunjun-connector-filesystem - chunjun-connector-ftp - - - - + chunjun-connector-ftp chunjun-connector-hbase-base chunjun-connector-hbase-1.4 - - + + chunjun-connector-elasticsearch7 chunjun-connector-elasticsearch-base chunjun-connector-redis @@ -96,6 +92,10 @@ ch.qos.logback logback-core + + org.apache.flink + flink-shaded-guava + @@ -131,4 +131,47 @@ + + + + default + + true + + + 2.7.5 + + + + chunjun-connector-hdfs + chunjun-connector-hive + + + + + + + tdh + + + chunjun-connector-inceptor + + + + + + + + hive3 + + 3.1.4 + + + + chunjun-connector-hive3 + + + + + diff --git a/chunjun-core/pom.xml b/chunjun-core/pom.xml index c8fe23e35c..d8a72a955d 100644 --- a/chunjun-core/pom.xml +++ b/chunjun-core/pom.xml @@ -1,116 +1,132 @@ - - chunjun - com.dtstack.chunjun - 1.12-SNAPSHOT - - 4.0.0 - - chunjun-core + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + chunjun + com.dtstack.chunjun + 1.12-SNAPSHOT + + 4.0.0 + + chunjun-core ChunJun : Core - - - - org.apache.logging.log4j - log4j-core - ${log4j2.version} + + + + org.apache.logging.log4j + log4j-core + ${log4j2.version} provided - + - - - - org.apache.logging.log4j - log4j-slf4j-impl + + + + org.apache.logging.log4j + log4j-slf4j-impl ${log4j2.version} provided - + + org.slf4j slf4j-api - 1.7.21 - provided + ${slf4j.version} + + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + + + + log4j + log4j + ${log4j.version} + ch.qos.logback logback-classic - 1.1.7 + ${logback.version} + provided ch.qos.logback logback-core - 1.1.7 + ${logback.version} + provided + + + + com.google.guava + guava + ${guava.version} + + + + com.google.code.gson + gson + 2.7 + + + + + org.apache.flink + flink-core + ${flink.version} + + + slf4j-api + org.slf4j + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + + + slf4j-api + org.slf4j + + - - com.google.guava - guava - - - - com.google.code.gson - gson - 2.7 - - - - - org.apache.flink - flink-core - ${flink.version} - - - slf4j-api - org.slf4j - - - - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${flink.version} - - - slf4j-api - org.slf4j - - - - - - org.apache.flink - flink-clients_${scala.binary.version} - ${flink.version} - - - slf4j-api - org.slf4j - - - - - - org.apache.flink - flink-runtime-web_${scala.binary.version} - ${flink.version} - - - slf4j-api - org.slf4j - - - + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + + + slf4j-api + org.slf4j + + + + + + org.apache.flink + flink-runtime-web_${scala.binary.version} + ${flink.version} + + + slf4j-api + org.slf4j + + + org.apache.flink @@ -124,71 +140,71 @@ - - - org.apache.flink - flink-table-planner-blink_${scala.binary.version} - ${flink.version} - - - slf4j-api - org.slf4j - - - - - - org.apache.flink - flink-table-api-java-bridge_${scala.binary.version} - ${flink.version} - - - slf4j-api - org.slf4j - - - - - - org.apache.flink - flink-statebackend-rocksdb_${scala.binary.version} - ${flink.version} - - - - org.apache.flink - flink-table-common - ${flink.version} - - - - - - - org.apache.flink - flink-hadoop-compatibility_${scala.binary.version} - ${flink.version} - - - org.xerial.snappy - snappy-java - - - - - - org.apache.flink - flink-yarn_${scala.binary.version} - ${flink.version} - - - flink-shaded-hadoop2 - org.apache.flink - - - slf4j-api - org.slf4j - + + + org.apache.flink + flink-table-planner-blink_${scala.binary.version} + ${flink.version} + + + slf4j-api + org.slf4j + + + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${flink.version} + + + slf4j-api + org.slf4j + + + + + + org.apache.flink + flink-statebackend-rocksdb_${scala.binary.version} + ${flink.version} + + + + org.apache.flink + flink-table-common + ${flink.version} + + + + + + + org.apache.flink + flink-hadoop-compatibility_${scala.binary.version} + ${flink.version} + + + org.xerial.snappy + snappy-java + + + + + + org.apache.flink + flink-yarn_${scala.binary.version} + ${flink.version} + + + flink-shaded-hadoop2 + org.apache.flink + + + slf4j-api + org.slf4j + org.apache.hadoop hadoop-common @@ -209,48 +225,48 @@ org.apache.hadoop hadoop-mapreduce-client-core - - - - - org.apache.flink - flink-queryable-state-runtime_${scala.binary.version} - ${flink.version} - - - - org.apache.httpcomponents - httpclient - ${http.version} - - - - com.fasterxml.jackson.core - jackson-databind - 2.9.10.1 - - - - commons-net - commons-net - 3.1 - - - - commons-lang - commons-lang - 2.6 - - - com.github.jsqlparser - jsqlparser - 3.2 - test - - - - - + + + + + org.apache.flink + flink-queryable-state-runtime_${scala.binary.version} + ${flink.version} + + + + org.apache.httpcomponents + httpclient + ${http.version} + + + + com.fasterxml.jackson.core + jackson-databind + 2.9.10.1 + + + + commons-net + commons-net + 3.1 + + + + commons-lang + commons-lang + 2.6 + + + com.github.jsqlparser + jsqlparser + 3.2 + test + + + + + net.alchim31.maven @@ -279,96 +295,91 @@ - - org.apache.maven.plugins - maven-compiler-plugin - - - - org.apache.maven.plugins - maven-shade-plugin - - - package - - shade - - - false - - - com.dtstack.chunjun.Main - - - - - com.google.guava:* - com.google.code.gson:* - ch.qos.logback:* - org.slf4j:* - org.apache.httpcomponents:* - io.prometheus:* - org.apache.avro:* - com.fasterxml.jackson.core:* - commons-*:* - - - - - com.google.common - shade.core.com.google.common - - - com.google.thirdparty - shade.core.com.google.thirdparty - - - org.apache.http - shade.core.org.apache.http - - - io.prometheus.client - shade.core.io.prometheus.client - - - - - - - - - maven-antrun-plugin - - - copy-resources - - package - - run - - - - + org.apache.maven.plugins + maven-compiler-plugin + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + false + + + com.dtstack.chunjun.Main + + + + + com.google.guava:* + com.google.code.gson:* + ch.qos.logback:* + org.slf4j:* + org.apache.httpcomponents:* + io.prometheus:* + org.apache.avro:* + com.fasterxml.jackson.core:* + commons-*:* + + + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + org.apache.http + shade.core.org.apache.http + + + io.prometheus.client + shade.core.io.prometheus.client + + + + + + + + + maven-antrun-plugin + + + copy-resources + + package + + run + + + + - - - - - - + + excludes="${project.artifactId}-${git.branch}.jar"/> - - - - - - - + + + + + + + diff --git a/chunjun-examples/json/hive3/hive3_orc_stream.json b/chunjun-examples/json/hive3/hive3_orc_stream.json new file mode 100644 index 0000000000..8843cb0c43 --- /dev/null +++ b/chunjun-examples/json/hive3/hive3_orc_stream.json @@ -0,0 +1,57 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "path": "hdfs://ns/user/hive/warehouse/dtstack1", + "hadoopConfig": { + "dfs.nameservices": "ns", + "fs.defaultFS": "hdfs://ns", + "dfs.ha.namenodes.ns": "nn1,nn2", + "dfs.namenode.rpc-address.ns.nn1": "hadoop3x1:9000", + "dfs.namenode.rpc-address.ns.nn2": "hadoop3x2:9000", + "dfs.client.failover.proxy.provider.ns": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "fs.hdfs.impl.disable.cache": "true", + "fs.hdfs.impl": "org.apache.hadoop.hdfs.DistributedFileSystem" + }, + "column": [ + { + "index": 0, + "name": "id", + "type": "int" + }, + { + "index": 1, + "name": "name", + "type": "String" + }, + { + "index": 2, + "name": "age", + "type": "int" + } + ], + "fs.defaultFS": "hdfs://ns", + "fieldDelimiter": ",", + "encoding": "utf-8", + "fileType": "orc" + }, + "name": "hive3reader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + } + } + } +} diff --git a/chunjun-examples/json/hive3/hive3_transaction_stream.json b/chunjun-examples/json/hive3/hive3_transaction_stream.json new file mode 100644 index 0000000000..0b82e69c6c --- /dev/null +++ b/chunjun-examples/json/hive3/hive3_transaction_stream.json @@ -0,0 +1,58 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "path": "hdfs://ns/user/hive/warehouse/dtstack", + "hadoopConfig": { + "dfs.nameservices": "ns", + "fs.defaultFS": "hdfs://ns", + "dfs.ha.namenodes.ns": "nn1,nn2", + "dfs.namenode.rpc-address.ns.nn1": "hadoop3x1:9000", + "dfs.namenode.rpc-address.ns.nn2": "hadoop3x2:9000", + "dfs.client.failover.proxy.provider.ns": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "fs.hdfs.impl.disable.cache": "true", + "fs.hdfs.impl": "org.apache.hadoop.hdfs.DistributedFileSystem" + }, + "column": [ + { + "index": 0, + "name": "id", + "type": "int" + }, + { + "index": 1, + "name": "name", + "type": "String" + }, + { + "index": 2, + "name": "age", + "type": "int" + } + ], + "defaultFS": "hdfs://ns", + "fieldDelimiter": ",", + "encoding": "utf-8", + "fileType": "orc", + "isTransaction": true + }, + "name": "hive3reader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + } + } + } +} diff --git a/chunjun-examples/json/hive3/stream_hive3_orc.json b/chunjun-examples/json/hive3/stream_hive3_orc.json new file mode 100644 index 0000000000..b90d8e3500 --- /dev/null +++ b/chunjun-examples/json/hive3/stream_hive3_orc.json @@ -0,0 +1,79 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": [ + 10 + ], + "column": [ + { + "index": 0, + "name": "id", + "type": "int" + }, + { + "index": 1, + "name": "name", + "type": "String" + }, + { + "index": 2, + "name": "age", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "hive3writer", + "parameter": { + "path": "hdfs://ns/user/hive/warehouse/dtstack1", + "defaultFS": "hdfs://ns", + "fieldDelimiter": ",", + "encoding": "utf-8", + "fileType": "orc", + "maxFileSize": 10485760, + "nextCheckRows": 20000, + "writeMode": "overwrite", + "hadoopConfig": { + "dfs.nameservices": "ns", + "fs.defaultFS": "hdfs://ns", + "dfs.ha.namenodes.ns": "nn1,nn2", + "dfs.namenode.rpc-address.ns.nn1": "hadoop3x1:9000", + "dfs.namenode.rpc-address.ns.nn2": "hadoop3x2:9000", + "dfs.client.failover.proxy.provider.ns": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "fs.hdfs.impl.disable.cache": "true", + "fs.hdfs.impl": "org.apache.hadoop.hdfs.DistributedFileSystem" + }, + "column": [ + { + "index": 0, + "name": "id", + "type": "int" + }, + { + "index": 1, + "name": "name", + "type": "String" + }, + { + "index": 2, + "name": "age", + "type": "int" + } + ] + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + } + } + } +} diff --git a/chunjun-examples/json/hive3/stream_hive3_transaction.json b/chunjun-examples/json/hive3/stream_hive3_transaction.json new file mode 100644 index 0000000000..f6ab466256 --- /dev/null +++ b/chunjun-examples/json/hive3/stream_hive3_transaction.json @@ -0,0 +1,89 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": [ + 10 + ], + "column": [ + { + "index": 0, + "name": "id", + "type": "int" + }, + { + "index": 1, + "name": "name", + "type": "String" + }, + { + "index": 2, + "name": "age", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "hive3writer", + "parameter": { + "table": "dtstack", + "schema": "default", + "isTransaction": true, + "path": "hdfs://ns/user/hive/warehouse/dtstack", + "defaultFS": "hdfs://ns", + "fieldDelimiter": ",", + "encoding": "utf-8", + "fileType": "ORC", + "maxFileSize": 10485760, + "nextCheckRows": 20000, + "writeMode": "append", + "hadoopConfig": { + "dfs.nameservices": "ns", + "fs.defaultFS": "hdfs://ns", + "dfs.ha.namenodes.ns": "nn1,nn2", + "dfs.namenode.rpc-address.ns.nn1": "hadoop3x1:9000", + "dfs.namenode.rpc-address.ns.nn2": "hadoop3x2:9000", + "dfs.client.failover.proxy.provider.ns": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "fs.hdfs.impl.disable.cache": "true", + "fs.hdfs.impl": "org.apache.hadoop.hdfs.DistributedFileSystem", + "hive.metastore.uris": "thrift://hadoop3x1:9083", + "hadoop.user.name": "root", + "hive.metastore.warehouse.dir": "/warehouse/tablespace/managed/hive", + "hive.metastore.warehouse.external.dir": "/warehouse/tablespace/external/hive", + "hive.metastore.execute.setugi": "true", + "hive.metastore.dml.events": "true", + "hive.metastore.transactional.event.listeners": "org.apache.hive.hcatalog.listener.DbNotificationListener" + }, + "column": [ + { + "index": 0, + "name": "id", + "type": "int" + }, + { + "index": 1, + "name": "name", + "type": "String" + }, + { + "index": 2, + "name": "age", + "type": "int" + } + ] + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + } + } + } +} diff --git a/chunjun-restore/chunjun-restore-common/pom.xml b/chunjun-restore/chunjun-restore-common/pom.xml index 57481227b4..1a285ffb83 100644 --- a/chunjun-restore/chunjun-restore-common/pom.xml +++ b/chunjun-restore/chunjun-restore-common/pom.xml @@ -22,20 +22,19 @@ org.slf4j slf4j-simple 1.7.32 + provided - log4j log4j - 1.2.17 + ${log4j.version} + provided - com.dtstack.chunjun chunjun-core ${project.version} - diff --git a/pom.xml b/pom.xml index 0219373965..095d4090af 100644 --- a/pom.xml +++ b/pom.xml @@ -14,7 +14,6 @@ chunjun-core chunjun-connectors chunjun-clients - chunjun-formats chunjun-docker chunjun-metrics @@ -30,13 +29,16 @@ 1.12.7 2.12 0.8.1 - 2.7.5 + 3.1.4 4.5.3 - 19.0 + 27.0-jre 2.17.0 chunjun-dist 2.0.4 + 1.2.17 + 1.7.36 + 1.2.11 @@ -97,6 +99,10 @@ commons-cli commons-cli + + guava + com.google.guava + @@ -184,6 +190,10 @@ commons-cli commons-cli + + guava + com.google.guava + @@ -199,6 +209,12 @@ hadoop-yarn-api ${hadoop.version} provided + + + guava + com.google.guava + + @@ -207,17 +223,16 @@ 1.8.2 provided - - - - - com.google.guava - guava - ${guava.version} - - - + + + + + + + + + @@ -379,7 +394,6 @@ - local-test @@ -387,47 +401,5 @@ chunjun-local-test - - - - default - - true - - - 2.7.5 - - - - chunjun-connectors/chunjun-connector-hdfs - chunjun-connectors/chunjun-connector-hive - - - - - - - tdh - - - chunjun-connectors/chunjun-connector-inceptor - - - - - - - - hive3 - - 3.1.4 - - - - chunjun-connectors/chunjun-connector-hive3 - - - - From 7dcfe537b7ad84a2ad9b58e014f23600ed11f2f6 Mon Sep 17 00:00:00 2001 From: wujuan Date: Mon, 8 Aug 2022 10:13:27 +0800 Subject: [PATCH 3/3] [feat][hive3] The version of guava conflict lead compile fail. --- .../chunjun/connector/doris/rest/FeRestService.java | 12 ------------ chunjun-local-test/pom.xml | 10 +++++----- 2 files changed, 5 insertions(+), 17 deletions(-) diff --git a/chunjun-connectors/chunjun-connector-doris/src/main/java/com/dtstack/chunjun/connector/doris/rest/FeRestService.java b/chunjun-connectors/chunjun-connector-doris/src/main/java/com/dtstack/chunjun/connector/doris/rest/FeRestService.java index 5c826c7c29..e1226bc634 100644 --- a/chunjun-connectors/chunjun-connector-doris/src/main/java/com/dtstack/chunjun/connector/doris/rest/FeRestService.java +++ b/chunjun-connectors/chunjun-connector-doris/src/main/java/com/dtstack/chunjun/connector/doris/rest/FeRestService.java @@ -28,8 +28,6 @@ import com.dtstack.chunjun.connector.doris.rest.module.Schema; import com.dtstack.chunjun.connector.doris.rest.module.Tablet; -import org.apache.flink.shaded.guava18.com.google.common.annotations.VisibleForTesting; - import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -242,7 +240,6 @@ private static String parseResponse(HttpURLConnection connection) throws IOExcep * @return first element is db name, second element is table name * @throws IllegalArgumentException table identifier is illegal */ - @VisibleForTesting static String[] parseIdentifier(String tableIdentifier) throws IllegalArgumentException { LOG.trace("Parse identifier '{}'.", tableIdentifier); if (StringUtils.isEmpty(tableIdentifier)) { @@ -264,7 +261,6 @@ static String[] parseIdentifier(String tableIdentifier) throws IllegalArgumentEx * @return the chosen one Doris FE node * @throws IllegalArgumentException fe nodes is illegal */ - @VisibleForTesting static String randomEndpoint(List feNodes) throws IllegalArgumentException { LOG.trace("Parse feNodes '{}'.", feNodes); if (feNodes.isEmpty()) { @@ -282,7 +278,6 @@ static String randomEndpoint(List feNodes) throws IllegalArgumentExcepti * @return the chosen one Doris BE node * @throws IllegalArgumentException BE nodes is illegal */ - @VisibleForTesting public static String randomBackend(DorisConf options) throws IOException { List backends = getBackends(options); LOG.trace("Parse beNodes '{}'.", backends); @@ -302,7 +297,6 @@ public static String randomBackend(DorisConf options) throws IOException { * @return the chosen one Doris BE node * @throws IllegalArgumentException BE nodes is illegal */ - @VisibleForTesting static List getBackends(DorisConf options) throws IOException { List feNodes = options.getFeNodes(); String feNode = randomEndpoint(feNodes); @@ -351,7 +345,6 @@ static List parseBackend(String response) throws IOException { * @return uri string * @throws IllegalArgumentException throw when configuration is illegal */ - @VisibleForTesting static String getUriStr(DorisConf options) throws IllegalArgumentException { return "http://" + randomEndpoint(options.getFeNodes()) @@ -385,7 +378,6 @@ public static Schema getSchema(DorisConf options) throws RuntimeException { * @return inner {@link Schema} struct * @throws RuntimeException throw when translate failed */ - @VisibleForTesting public static Schema parseSchema(String response) throws RuntimeException { LOG.trace("Parse response '{}' to schema.", response); ObjectMapper mapper = new ObjectMapper(); @@ -472,7 +464,6 @@ public static List findPartitions(DorisConf options) * @return inner {@link QueryPlan} struct * @throws RuntimeException throw when translate failed. */ - @VisibleForTesting static QueryPlan getQueryPlan(String response) throws RuntimeException { ObjectMapper mapper = new ObjectMapper(); QueryPlan queryPlan; @@ -513,7 +504,6 @@ static QueryPlan getQueryPlan(String response) throws RuntimeException { * @return BE to tablets {@link Map} * @throws RuntimeException throw when select failed. */ - @VisibleForTesting static Map> selectBeForTablet(QueryPlan queryPlan) throws RuntimeException { Map> be2Tablets = new HashMap<>(); for (Map.Entry part : queryPlan.getPartitions().entrySet()) { @@ -566,7 +556,6 @@ static Map> selectBeForTablet(QueryPlan queryPlan) throws Run * @param loadConf configuration of request * @return tablet count limit */ - @VisibleForTesting static int tabletCountLimitForOnePartition(LoadConf loadConf) { int tabletsSize = DORIS_TABLET_SIZE_DEFAULT; if (loadConf.getRequestTabletSize() != null) { @@ -595,7 +584,6 @@ static int tabletCountLimitForOnePartition(LoadConf loadConf) { * @return Doris RDD partition {@link List} * @throws IllegalArgumentException throw when translate failed */ - @VisibleForTesting static List tabletsMapToPartition( DorisConf options, Map> be2Tablets, diff --git a/chunjun-local-test/pom.xml b/chunjun-local-test/pom.xml index 2f3df67682..e85c149757 100644 --- a/chunjun-local-test/pom.xml +++ b/chunjun-local-test/pom.xml @@ -206,11 +206,11 @@ chunjun-connector-saphana ${project.version} - - com.dtstack.chunjun - chunjun-connector-inceptor - ${project.version} - + + + + + com.dtstack.chunjun chunjun-connector-oceanbase