diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/pom.xml b/chunjun-connectors/chunjun-connector-hbase-1.4/pom.xml index 1e77e74b03..5bd55031f7 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/pom.xml +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/pom.xml @@ -41,6 +41,12 @@ com.dtstack.chunjun chunjun-connector-hbase-base ${project.version} + + + log4j-1.2-api + org.apache.logging.log4j + + org.hbase diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/AsyncHBaseSerde.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/AsyncHBaseSerde.java index 41a8a95b74..226ecaa090 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/AsyncHBaseSerde.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/AsyncHBaseSerde.java @@ -18,7 +18,6 @@ package com.dtstack.chunjun.connector.hbase14.converter; -import com.dtstack.chunjun.connector.hbase.HBaseSerde; import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; import org.apache.flink.table.data.GenericRowData; @@ -29,7 +28,7 @@ import java.util.Map; /** - * @program chunjun + * @program: flinkx * @author: wuren * @create: 2021/10/19 */ diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/DataSyncConverterUtils.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/DataSyncConverterUtils.java deleted file mode 100644 index c5e6894303..0000000000 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/DataSyncConverterUtils.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.chunjun.connector.hbase14.converter; - -import com.dtstack.chunjun.enums.ColumnType; - -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.util.Bytes; - -import java.nio.charset.Charset; -import java.sql.Timestamp; - -/** - * @program chunjun - * @author: wuren - * @create: 2021/10/19 - */ -public class DataSyncConverterUtils { - - public static byte[] getValueByte(ColumnType columnType, String value, String encoding) { - byte[] bytes; - if (value != null) { - switch (columnType) { - case INT: - bytes = Bytes.toBytes(Integer.parseInt(value)); - break; - case LONG: - bytes = Bytes.toBytes(Long.parseLong(value)); - break; - case DOUBLE: - bytes = Bytes.toBytes(Double.parseDouble(value)); - break; - case FLOAT: - bytes = Bytes.toBytes(Float.parseFloat(value)); - break; - case SHORT: - bytes = Bytes.toBytes(Short.parseShort(value)); - break; - case BOOLEAN: - bytes = Bytes.toBytes(Boolean.parseBoolean(value)); - break; - case STRING: - bytes = value.getBytes(Charset.forName(encoding)); - break; - default: - throw new IllegalArgumentException("Unsupported column type: " + columnType); - } - } else { - bytes = HConstants.EMPTY_BYTE_ARRAY; - } - return bytes; - } - - public static byte[] intToBytes(Object column) { - Integer intValue = null; - if (column instanceof Integer) { - intValue = (Integer) column; - } else if (column instanceof Long) { - intValue = ((Long) column).intValue(); - } else if (column instanceof Double) { - intValue = ((Double) column).intValue(); - } else if (column instanceof Float) { - intValue = ((Float) column).intValue(); - } else if (column instanceof Short) { - intValue = ((Short) column).intValue(); - } else if (column instanceof Boolean) { - intValue = (Boolean) column ? 1 : 0; - } else if (column instanceof String) { - intValue = Integer.valueOf((String) column); - } else { - throw new RuntimeException("Can't convert from " + column.getClass() + " to INT"); - } - - return Bytes.toBytes(intValue); - } - - public static byte[] longToBytes(Object column) { - Long longValue = null; - if (column instanceof Integer) { - longValue = ((Integer) column).longValue(); - } else if (column instanceof Long) { - longValue = (Long) column; - } else if (column instanceof Double) { - longValue = ((Double) column).longValue(); - } else if (column instanceof Float) { - longValue = ((Float) column).longValue(); - } else if (column instanceof Short) { - longValue = ((Short) column).longValue(); - } else if (column instanceof Boolean) { - longValue = (Boolean) column ? 1L : 0L; - } else if (column instanceof String) { - longValue = Long.valueOf((String) column); - } else if (column instanceof Timestamp) { - longValue = ((Timestamp) column).getTime(); - } else { - throw new RuntimeException("Can't convert from " + column.getClass() + " to LONG"); - } - - return Bytes.toBytes(longValue); - } - - public static byte[] doubleToBytes(Object column) { - Double doubleValue; - if (column instanceof Integer) { - doubleValue = ((Integer) column).doubleValue(); - } else if (column instanceof Long) { - doubleValue = ((Long) column).doubleValue(); - } else if (column instanceof Double) { - doubleValue = (Double) column; - } else if (column instanceof Float) { - doubleValue = ((Float) column).doubleValue(); - } else if (column instanceof Short) { - doubleValue = ((Short) column).doubleValue(); - } else if (column instanceof Boolean) { - doubleValue = (Boolean) column ? 1.0 : 0.0; - } else if (column instanceof String) { - doubleValue = Double.valueOf((String) column); - } else { - throw new RuntimeException("Can't convert from " + column.getClass() + " to DOUBLE"); - } - - return Bytes.toBytes(doubleValue); - } - - public static byte[] floatToBytes(Object column) { - Float floatValue = null; - if (column instanceof Integer) { - floatValue = ((Integer) column).floatValue(); - } else if (column instanceof Long) { - floatValue = ((Long) column).floatValue(); - } else if (column instanceof Double) { - floatValue = ((Double) column).floatValue(); - } else if (column instanceof Float) { - floatValue = (Float) column; - } else if (column instanceof Short) { - floatValue = ((Short) column).floatValue(); - } else if (column instanceof Boolean) { - floatValue = (Boolean) column ? 1.0f : 0.0f; - } else if (column instanceof String) { - floatValue = Float.valueOf((String) column); - } else { - throw new RuntimeException("Can't convert from " + column.getClass() + " to DOUBLE"); - } - - return Bytes.toBytes(floatValue); - } - - public static byte[] shortToBytes(Object column) { - Short shortValue = null; - if (column instanceof Integer) { - shortValue = ((Integer) column).shortValue(); - } else if (column instanceof Long) { - shortValue = ((Long) column).shortValue(); - } else if (column instanceof Double) { - shortValue = ((Double) column).shortValue(); - } else if (column instanceof Float) { - shortValue = ((Float) column).shortValue(); - } else if (column instanceof Short) { - shortValue = (Short) column; - } else if (column instanceof Boolean) { - shortValue = (Boolean) column ? (short) 1 : (short) 0; - } else if (column instanceof String) { - shortValue = Short.valueOf((String) column); - } else { - throw new RuntimeException("Can't convert from " + column.getClass() + " to SHORT"); - } - return Bytes.toBytes(shortValue); - } - - public static byte[] boolToBytes(Object column) { - Boolean booleanValue = null; - if (column instanceof Integer) { - booleanValue = (Integer) column != 0; - } else if (column instanceof Long) { - booleanValue = (Long) column != 0L; - } else if (column instanceof Double) { - booleanValue = new Double(0.0).compareTo((Double) column) != 0; - } else if (column instanceof Float) { - booleanValue = new Float(0.0f).compareTo((Float) column) != 0; - } else if (column instanceof Short) { - booleanValue = (Short) column != 0; - } else if (column instanceof Boolean) { - booleanValue = (Boolean) column; - } else if (column instanceof String) { - booleanValue = Boolean.valueOf((String) column); - } else { - throw new RuntimeException("Can't convert from " + column.getClass() + " to SHORT"); - } - - return Bytes.toBytes(booleanValue); - } -} diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/DataSyncSinkConverter.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/DataSyncSinkConverter.java deleted file mode 100644 index b850de4437..0000000000 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/DataSyncSinkConverter.java +++ /dev/null @@ -1,336 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.chunjun.connector.hbase14.converter; - -import com.dtstack.chunjun.connector.hbase14.sink.FunctionParser; -import com.dtstack.chunjun.connector.hbase14.sink.FunctionTree; -import com.dtstack.chunjun.constants.ConstantValue; -import com.dtstack.chunjun.element.ColumnRowData; -import com.dtstack.chunjun.element.column.BigDecimalColumn; -import com.dtstack.chunjun.enums.ColumnType; -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 com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.util.Bytes; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Serializable; -import java.nio.charset.StandardCharsets; -import java.sql.Timestamp; -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * @program chunjun - * @author: wuren - * @create: 2021/10/19 - */ -public class DataSyncSinkConverter implements Serializable { - - private static final Logger LOG = LoggerFactory.getLogger(DataSyncSinkConverter.class); - - private boolean walFlag; - private String nullMode; - private String encoding; - - private List columnTypes; - private List columnNames; - private Integer versionColumnIndex; - - private String versionColumnValue; - private List rowKeyColumns; - private List rowKeyColumnIndex; - - private transient FunctionTree functionTree; - private transient Map nameMaps; - private transient Map nameByteMaps; - private transient ThreadLocal timeSecondFormatThreadLocal; - private transient ThreadLocal timeMillisecondFormatThreadLocal; - - public DataSyncSinkConverter( - boolean walFlag, - String nullMode, - String encoding, - List columnTypes, - List columnNames, - String rowkeyExpress, - Integer versionColumnIndex, - String versionColumnValue) { - this.walFlag = walFlag; - this.nullMode = nullMode; - this.encoding = encoding; - this.columnTypes = columnTypes; - this.columnNames = columnNames; - this.versionColumnIndex = versionColumnIndex; - this.versionColumnValue = versionColumnValue; - - this.rowKeyColumns = Lists.newArrayList(); - this.rowKeyColumnIndex = Lists.newArrayList(); - - this.nameMaps = Maps.newConcurrentMap(); - this.nameByteMaps = Maps.newConcurrentMap(); - timeSecondFormatThreadLocal = new ThreadLocal(); - timeMillisecondFormatThreadLocal = new ThreadLocal(); - - this.functionTree = FunctionParser.parse(rowkeyExpress); - this.rowKeyColumns = FunctionParser.parseRowKeyCol(rowkeyExpress); - for (String rowKeyColumn : rowKeyColumns) { - int index = columnNames.indexOf(rowKeyColumn); - if (index == -1) { - throw new RuntimeException( - "Can not get row key column from columns:" + rowKeyColumn); - } - rowKeyColumnIndex.add(index); - } - } - - public Put generatePutCommand(RowData rowData) throws WriteRecordException { - int i = 0; - try { - byte[] rowkey = getRowkey(rowData); - Put put; - if (versionColumnIndex == null) { - put = new Put(rowkey); - if (!walFlag) { - put.setDurability(Durability.SKIP_WAL); - } - } else { - long timestamp = getVersion(rowData); - put = new Put(rowkey, timestamp); - } - - for (; i < rowData.getArity(); ++i) { - if (rowKeyColumnIndex.contains(i)) { - continue; - } - - String type = columnTypes.get(i); - String name = columnNames.get(i); - String[] cfAndQualifier = nameMaps.get(name); - byte[][] cfAndQualifierBytes = nameByteMaps.get(name); - if (cfAndQualifier == null || cfAndQualifierBytes == null) { - cfAndQualifier = name.split(":"); - if (cfAndQualifier.length == 2 - && StringUtils.isNotBlank(cfAndQualifier[0]) - && StringUtils.isNotBlank(cfAndQualifier[1])) { - nameMaps.put(name, cfAndQualifier); - cfAndQualifierBytes = new byte[2][]; - cfAndQualifierBytes[0] = Bytes.toBytes(cfAndQualifier[0]); - cfAndQualifierBytes[1] = Bytes.toBytes(cfAndQualifier[1]); - nameByteMaps.put(name, cfAndQualifierBytes); - } else { - throw new IllegalArgumentException( - "Hbasewriter 中,column 的列配置格式应该是:列族:列名. 您配置的列错误:" + name); - } - } - - ColumnType columnType = ColumnType.getType(type); - Object column = null; - if (rowData instanceof GenericRowData) { - column = ((GenericRowData) rowData).getField(i); - } else if (rowData instanceof ColumnRowData) { - column = ((ColumnRowData) rowData).getField(i); - } - byte[] columnBytes = getColumnByte(columnType, column); - // columnBytes 为null忽略这列 - if (null != columnBytes) { - put.addColumn(cfAndQualifierBytes[0], cfAndQualifierBytes[1], columnBytes); - } - } - return put; - } catch (Exception ex) { - if (i < rowData.getArity()) { - throw new WriteRecordException(ex.getMessage(), ex, i, rowData); - } - throw new WriteRecordException(ex.getMessage(), ex); - } - } - - private byte[] getRowkey(RowData record) throws Exception { - Map nameValueMap = new HashMap<>((rowKeyColumnIndex.size() << 2) / 3); - for (Integer keyColumnIndex : rowKeyColumnIndex) { - Object column = null; - if (record instanceof GenericRowData) { - column = ((GenericRowData) record).getField(keyColumnIndex); - } else if (record instanceof ColumnRowData) { - column = ((ColumnRowData) record).getField(keyColumnIndex); - } - nameValueMap.put(columnNames.get(keyColumnIndex), column); - } - - String rowKeyStr = functionTree.evaluate(nameValueMap); - return rowKeyStr.getBytes(StandardCharsets.UTF_8); - } - - public long getVersion(RowData rawRecord) { - RowData record = rawRecord; - Integer index = versionColumnIndex.intValue(); - long timestamp; - if (index == null) { - // 指定时间作为版本 - timestamp = Long.valueOf(versionColumnValue); - if (timestamp < 0) { - throw new IllegalArgumentException( - "Illegal timestamp to construct versionClumn: " + timestamp); - } - } else { - // 指定列作为版本,long/doubleColumn直接record.aslong, 其它类型尝试用yyyy-MM-dd HH:mm:ss,yyyy-MM-dd - // HH:mm:ss SSS去format - if (index >= record.getArity() || index < 0) { - throw new IllegalArgumentException("version column index out of range: " + index); - } - Object column = null; - if (record instanceof GenericRowData) { - column = ((GenericRowData) record).getField(index); - } else if (record instanceof ColumnRowData) { - column = ((ColumnRowData) record).getField(index); - } - if (column == null) { - throw new IllegalArgumentException("null verison column!"); - } - SimpleDateFormat dfSeconds = getSimpleDateFormat(ConstantValue.TIME_SECOND_SUFFIX); - SimpleDateFormat dfMs = getSimpleDateFormat(ConstantValue.TIME_MILLISECOND_SUFFIX); - - if (column instanceof Long) { - Long longValue = (Long) column; - timestamp = longValue; - } else if (column instanceof Double) { - Double doubleValue = (Double) column; - timestamp = doubleValue.longValue(); - } else if (column instanceof String) { - Date date; - try { - - date = dfMs.parse((String) column); - } catch (ParseException e) { - try { - date = dfSeconds.parse((String) column); - } catch (ParseException e1) { - LOG.info( - String.format( - "您指定第[%s]列作为hbase写入版本,但在尝试用yyyy-MM-dd HH:mm:ss 和 yyyy-MM-dd HH:mm:ss SSS 去解析为Date时均出错,请检查并修改", - index)); - throw new RuntimeException(e1); - } - } - timestamp = date.getTime(); - } else if (column instanceof Date) { - timestamp = ((Date) column).getTime(); - } else if (column instanceof BigDecimalColumn) { - timestamp = ((BigDecimalColumn) column).asLong(); - } else { - throw new RuntimeException("rowkey类型不兼容: " + column.getClass()); - } - } - return timestamp; - } - - public byte[] getColumnByte(ColumnType columnType, Object column) { - byte[] bytes; - if (column != null) { - switch (columnType) { - case INT: - bytes = DataSyncConverterUtils.intToBytes(column); - break; - case LONG: - bytes = DataSyncConverterUtils.longToBytes(column); - break; - case DOUBLE: - bytes = DataSyncConverterUtils.doubleToBytes(column); - break; - case FLOAT: - bytes = DataSyncConverterUtils.floatToBytes(column); - break; - case SHORT: - bytes = DataSyncConverterUtils.shortToBytes(column); - break; - case BOOLEAN: - bytes = DataSyncConverterUtils.boolToBytes(column); - break; - case STRING: - String stringValue; - if (column instanceof Timestamp) { - SimpleDateFormat fm = DateUtil.getDateTimeFormatter(); - stringValue = fm.format(column); - } else { - stringValue = String.valueOf(column); - } - bytes = DataSyncConverterUtils.getValueByte(columnType, stringValue, encoding); - break; - default: - throw new IllegalArgumentException("Unsupported column type: " + columnType); - } - } else { - switch (nullMode.toUpperCase()) { - case "SKIP": - bytes = null; - break; - case "EMPTY": - bytes = HConstants.EMPTY_BYTE_ARRAY; - break; - default: - throw new IllegalArgumentException("Unsupported null mode: " + nullMode); - } - } - return bytes; - } - - private SimpleDateFormat getSimpleDateFormat(String sign) { - SimpleDateFormat format; - if (ConstantValue.TIME_SECOND_SUFFIX.equals(sign)) { - format = timeSecondFormatThreadLocal.get(); - if (format == null) { - format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - timeSecondFormatThreadLocal.set(format); - } - } else { - format = timeMillisecondFormatThreadLocal.get(); - if (format == null) { - format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss SSS"); - timeMillisecondFormatThreadLocal.set(format); - } - } - - return format; - } - - public void close() { - if (null != timeSecondFormatThreadLocal) { - timeSecondFormatThreadLocal.remove(); - } - - if (null != timeMillisecondFormatThreadLocal) { - timeMillisecondFormatThreadLocal.remove(); - } - } -} diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseColumnConverter.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseColumnConverter.java new file mode 100644 index 0000000000..473dcd6401 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseColumnConverter.java @@ -0,0 +1,514 @@ +/* + * 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.hbase14.converter; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hbase.FunctionParser; +import com.dtstack.chunjun.connector.hbase.FunctionTree; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.converter.type.BINARYSTRING; +import com.dtstack.chunjun.constants.ConstantValue; +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.BytesColumn; +import com.dtstack.chunjun.element.column.SqlDateColumn; +import com.dtstack.chunjun.element.column.StringColumn; +import com.dtstack.chunjun.element.column.TimeColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; + +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.flink.table.types.logical.TimestampType; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.time.DateUtils; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class HBaseColumnConverter + extends AbstractRowConverter { + + public static final String KEY_ROW_KEY = "rowkey"; + + private final HBaseConf hBaseConf; + private final List fieldList; + private final List columnNames; + + // sink + private final boolean walFlag; + // qualifier keys + private final byte[][][] familyAndQualifier; + private final String encoding; + private final String nullMode; + private List rowKeyColumnIndex; + private List rowKeyColumns; + private final Integer versionColumnIndex; + private final String versionColumnValue; + private final SimpleDateFormat timeSecondFormat; + private final SimpleDateFormat timeMillisecondFormat; + + private FunctionTree functionTree; + + public HBaseColumnConverter(HBaseConf hBaseConf, RowType rowType) { + super(rowType); + this.hBaseConf = hBaseConf; + this.fieldList = hBaseConf.getColumn(); + this.encoding = hBaseConf.getEncoding(); + this.nullMode = hBaseConf.getNullMode(); + this.versionColumnIndex = hBaseConf.getVersionColumnIndex(); + this.versionColumnValue = hBaseConf.getVersionColumnValue(); + this.walFlag = hBaseConf.getWalFlag(); + this.familyAndQualifier = new byte[hBaseConf.getColumn().size()][][]; + this.columnNames = new ArrayList<>(hBaseConf.getColumn().size()); + for (int i = 0; i < hBaseConf.getColumn().size(); i++) { + String name = hBaseConf.getColumn().get(i).getName(); + columnNames.add(name); + String[] cfAndQualifier = name.split(":"); + if (cfAndQualifier.length == 2 + && org.apache.commons.lang.StringUtils.isNotBlank(cfAndQualifier[0]) + && org.apache.commons.lang.StringUtils.isNotBlank(cfAndQualifier[1])) { + + byte[][] qualifierKeys = new byte[2][]; + qualifierKeys[0] = Bytes.toBytes(cfAndQualifier[0]); + qualifierKeys[1] = Bytes.toBytes(cfAndQualifier[1]); + familyAndQualifier[i] = qualifierKeys; + } else if (!KEY_ROW_KEY.equals(name)) { + throw new IllegalArgumentException( + "hbase 中,column 的列配置格式应该是:列族:列名. 您配置的列错误:" + name); + } + } + + initRowKeyConfig(); + + for (int i = 0; i < hBaseConf.getColumn().size(); i++) { + + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(fieldTypes[i]), fieldTypes[i])); + } + this.timeSecondFormat = getSimpleDateFormat(ConstantValue.TIME_SECOND_SUFFIX); + this.timeMillisecondFormat = getSimpleDateFormat(ConstantValue.TIME_MILLISECOND_SUFFIX); + } + + @Override + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter ISerializationConverter, LogicalType type) { + return ((rowData, index, mutation) -> { + if (rowData != null && !rowData.isNullAt(index)) { + ISerializationConverter.serialize(rowData, index, mutation); + } else { + switch (nullMode.toUpperCase()) { + case "SKIP": + return; + case "EMPTY": + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + HConstants.EMPTY_BYTE_ARRAY); + return; + default: + throw new IllegalArgumentException("Unsupported null mode: " + nullMode); + } + } + }); + } + + @Override + public RowData toInternal(Result input) throws Exception { + ColumnRowData result = new ColumnRowData(fieldList.size()); + for (int i = 0; i < fieldList.size(); i++) { + AbstractBaseColumn baseColumn = null; + if (StringUtils.isBlank(fieldList.get(i).getValue())) { + byte[] bytes; + if (KEY_ROW_KEY.equals(fieldList.get(i).getName())) { + bytes = input.getRow(); + } else { + bytes = input.getValue(familyAndQualifier[i][0], familyAndQualifier[i][1]); + } + baseColumn = (AbstractBaseColumn) toInternalConverters.get(i).deserialize(bytes); + } + result.addField(assembleFieldProps(fieldList.get(i), baseColumn)); + } + + return result; + } + + @Override + public Mutation toExternal(RowData rowData, Mutation output) throws Exception { + byte[] rowkey = getRowkey(rowData); + Long version = getVersion(rowData); + Put put; + if (version == null) { + put = new Put(rowkey); + if (!walFlag) { + put.setDurability(Durability.SKIP_WAL); + } + } else { + put = new Put(rowkey, version); + } + + for (int i = 0; i < fieldList.size(); i++) { + if (rowKeyColumnIndex.contains(i)) { + continue; + } + this.toExternalConverters.get(i).serialize(rowData, i, put); + } + + return put; + } + + /** + * 将外部数据库类型转换为flink内部类型 + * + * @param type type + * @return return + */ + @Override + protected IDeserializationConverter createInternalConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + if (type instanceof BINARYSTRING) { + return val -> new StringColumn(Bytes.toStringBinary((byte[]) val)); + } + return val -> new StringColumn(new String((byte[]) val, encoding)); + case BOOLEAN: + return val -> { + // from flink + if (((byte[]) val).length == 1) { + return new BooleanColumn(((byte[]) val)[0] != 0); + } else { + return new BooleanColumn(Boolean.parseBoolean(val.toString())); + } + }; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return val -> + new TimestampColumn( + new BigDecimal(new String((byte[]) val, encoding)).longValue()); + case DECIMAL: + case INTEGER: + case INTERVAL_YEAR_MONTH: + case INTERVAL_DAY_TIME: + case FLOAT: + case DOUBLE: + return val -> { + try { + return new BigDecimalColumn(Bytes.toDouble((byte[]) val)); + } catch (Exception e) { + return new BigDecimalColumn(new String((byte[]) val, encoding)); + } + }; + case BIGINT: + return val -> { + try { + return new BigDecimalColumn(Bytes.toLong((byte[]) val)); + } catch (Exception e) { + return new BigDecimalColumn(new String((byte[]) val, encoding)); + } + }; + case TINYINT: + return val -> new BigDecimalColumn(((byte[]) val)[0]); + case SMALLINT: + return val -> new BigDecimalColumn(Bytes.toShort(((byte[]) val))); + case TIME_WITHOUT_TIME_ZONE: + return val -> new TimeColumn(Bytes.toInt(((byte[]) val))); + case BINARY: + case VARBINARY: + return val -> new BytesColumn(((byte[]) val)); + case DATE: + return val -> { + Date date; + try { + date = new Date(Bytes.toInt(((byte[]) val))); + } catch (Exception e) { + String dateValue = Bytes.toStringBinary(((byte[]) val)); + date = DateUtils.parseDate(dateValue); + } + return new SqlDateColumn(date.getTime()); + }; + + case TIMESTAMP_WITH_TIME_ZONE: + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (IDeserializationConverter) + val -> + new TimestampColumn( + Bytes.toLong((byte[]) val), + ((TimestampType) (type)).getPrecision()); + default: + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } + + /** + * 将flink内部的数据类型转换为外部数据库系统类型 + * + * @param type type + * @return return + */ + @Override + protected ISerializationConverter createExternalConverter(LogicalType type) { + + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + // get the underlying UTF-8 bytes + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + ((ColumnRowData) rowData) + .getField(index) + .asString() + .getBytes(encoding)); + case BOOLEAN: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData) + .getField(index) + .asBoolean())); + case BINARY: + case VARBINARY: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + ((ColumnRowData) rowData).getField(index).asBinary()); + case DECIMAL: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData) + .getField(index) + .asBigDecimal())); + case TINYINT: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + new byte[] { + ((ColumnRowData) rowData) + .getField(index) + .asInt() + .byteValue() + }); + case SMALLINT: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData) + .getField(index) + .asShort())); + case INTEGER: + case DATE: + case INTERVAL_YEAR_MONTH: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData).getField(index).asInt())); + case TIME_WITHOUT_TIME_ZONE: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData) + .getField(index) + .asTime() + .getTime())); + case BIGINT: + case INTERVAL_DAY_TIME: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData) + .getField(index) + .asLong())); + case FLOAT: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData) + .getField(index) + .asFloat())); + case DOUBLE: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData) + .getField(index) + .asDouble())); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (rowData, index, mutation) -> + ((Put) mutation) + .addColumn( + familyAndQualifier[index][0], + familyAndQualifier[index][1], + Bytes.toBytes( + ((ColumnRowData) rowData) + .getField(index) + .asTimestamp() + .getTime())); + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } + + private byte[] getRowkey(RowData record) throws Exception { + Map nameValueMap = new HashMap<>((rowKeyColumnIndex.size() << 2) / 3); + for (Integer keyColumnIndex : rowKeyColumnIndex) { + nameValueMap.put( + columnNames.get(keyColumnIndex), + ((ColumnRowData) record).getField(keyColumnIndex).getData()); + } + + String rowKeyStr = functionTree.evaluate(nameValueMap); + return rowKeyStr.getBytes(StandardCharsets.UTF_8); + } + + public Long getVersion(RowData record) { + if (versionColumnIndex == null && StringUtils.isBlank(versionColumnValue)) { + return null; + } + + Object timeStampValue = versionColumnValue; + if (versionColumnIndex != null) { + // 指定列作为版本,long/doubleColumn直接record.aslong, 其它类型尝试用yyyy-MM-dd HH:mm:ss,yyyy-MM-dd + // HH:mm:ss SSS去format + if (versionColumnIndex >= record.getArity() || versionColumnIndex < 0) { + throw new IllegalArgumentException( + "version column index out of range: " + versionColumnIndex); + } + if (record.isNullAt(versionColumnIndex)) { + throw new IllegalArgumentException("null verison column!"); + } + + timeStampValue = ((ColumnRowData) record).getField(versionColumnIndex).getData(); + } + + if (timeStampValue instanceof Long) { + return (Long) timeStampValue; + } else if (timeStampValue instanceof Double) { + return ((Double) timeStampValue).longValue(); + } else if (timeStampValue instanceof String) { + + try { + return Long.valueOf(timeStampValue.toString()); + } catch (Exception e) { + // ignore + } + java.util.Date date; + try { + date = timeMillisecondFormat.parse(timeStampValue.toString()); + } catch (ParseException e) { + try { + date = timeSecondFormat.parse(timeStampValue.toString()); + } catch (ParseException e1) { + LOG.info( + String.format( + "您指定第[%s]列作为hbase写入版本,但在尝试用yyyy-MM-dd HH:mm:ss 和 yyyy-MM-dd HH:mm:ss SSS 去解析为Date时均出错,请检查并修改", + versionColumnIndex)); + throw new RuntimeException(e1); + } + } + return date.getTime(); + } else if (timeStampValue instanceof java.util.Date) { + return ((Date) timeStampValue).getTime(); + } else { + throw new RuntimeException("rowkey类型不兼容: " + timeStampValue.getClass()); + } + } + + private SimpleDateFormat getSimpleDateFormat(String sign) { + SimpleDateFormat format; + if (ConstantValue.TIME_SECOND_SUFFIX.equals(sign)) { + format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + + } else { + format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss SSS"); + } + return format; + } + + private void initRowKeyConfig() { + if (StringUtils.isNotBlank(hBaseConf.getRowkeyExpress())) { + this.functionTree = FunctionParser.parse(hBaseConf.getRowkeyExpress()); + this.rowKeyColumns = FunctionParser.parseRowKeyCol(hBaseConf.getRowkeyExpress()); + this.rowKeyColumnIndex = new ArrayList<>(rowKeyColumns.size()); + for (String rowKeyColumn : rowKeyColumns) { + int index = columnNames.indexOf(rowKeyColumn); + if (index == -1) { + throw new RuntimeException( + "Can not get row key column from columns:" + rowKeyColumn); + } + rowKeyColumnIndex.add(index); + } + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseSerde.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseSerde.java similarity index 88% rename from chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseSerde.java rename to chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseSerde.java index 9fbb99059d..e76ec5c2a0 100644 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseSerde.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseSerde.java @@ -16,7 +16,9 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase; +package com.dtstack.chunjun.connector.hbase14.converter; + +import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericRowData; @@ -47,16 +49,16 @@ import static org.apache.flink.util.Preconditions.checkArgument; /** Utilities for HBase serialization and deserialization. */ -public class HBaseSerde { +public class HBaseSerde implements Serializable { - protected static final byte[] EMPTY_BYTES = new byte[] {}; + private static final byte[] EMPTY_BYTES = new byte[] {}; - protected static final int MIN_TIMESTAMP_PRECISION = 0; - protected static final int MAX_TIMESTAMP_PRECISION = 3; - protected static final int MIN_TIME_PRECISION = 0; - protected static final int MAX_TIME_PRECISION = 3; + private static final int MIN_TIMESTAMP_PRECISION = 0; + private static final int MAX_TIMESTAMP_PRECISION = 3; + private static final int MIN_TIME_PRECISION = 0; + private static final int MAX_TIME_PRECISION = 3; - protected final byte[] nullStringBytes; + private final byte[] nullStringBytes; // row key index in output row protected final int rowkeyIndex; @@ -68,14 +70,14 @@ public class HBaseSerde { protected final int fieldLength; - protected GenericRowData reusedRow; - protected GenericRowData[] reusedFamilyRows; + private GenericRowData reusedRow; + private GenericRowData[] reusedFamilyRows; - protected final @Nullable FieldEncoder keyEncoder; + private final @Nullable FieldEncoder keyEncoder; protected final @Nullable FieldDecoder keyDecoder; - protected final FieldEncoder[][] qualifierEncoders; + private final FieldEncoder[][] qualifierEncoders; protected final FieldDecoder[][] qualifierDecoders; - protected final GenericRowData rowWithRowKey; + private final GenericRowData rowWithRowKey; public HBaseSerde(HBaseTableSchema hbaseSchema, final String nullStringLiteral) { this.families = hbaseSchema.getFamilyKeys(); @@ -223,43 +225,18 @@ public Get createGet(Object rowKey) { return get; } - /** - * Converts HBase {@link Result} into a new {@link RowData} instance. - * - *

Note: this method is thread-safe. - */ - public RowData convertToNewRow(Result result) { - // The output rows needs to be initialized each time - // to prevent the possibility of putting the output object into the cache. - GenericRowData resultRow = new GenericRowData(fieldLength); - GenericRowData[] familyRows = new GenericRowData[families.length]; - for (int f = 0; f < families.length; f++) { - familyRows[f] = new GenericRowData(qualifiers[f].length); - } - return convertToRow(result, resultRow, familyRows); - } - - /** - * Converts HBase {@link Result} into a reused {@link RowData} instance. - * - *

Note: this method is NOT thread-safe. - */ - public RowData convertToReusedRow(Result result) { - return convertToRow(result, reusedRow, reusedFamilyRows); - } - - protected RowData convertToRow( - Result result, GenericRowData resultRow, GenericRowData[] familyRows) { + /** Converts HBase {@link Result} into {@link RowData}. */ + public RowData convertToRow(Result result) { for (int i = 0; i < fieldLength; i++) { if (rowkeyIndex == i) { assert keyDecoder != null; Object rowkey = keyDecoder.decode(result.getRow()); - resultRow.setField(rowkeyIndex, rowkey); + reusedRow.setField(rowkeyIndex, rowkey); } else { int f = (rowkeyIndex != -1 && i > rowkeyIndex) ? i - 1 : i; // get family key byte[] familyKey = families[f]; - GenericRowData familyRow = familyRows[f]; + GenericRowData familyRow = reusedFamilyRows[f]; for (int q = 0; q < this.qualifiers[f].length; q++) { // get quantifier key byte[] qualifier = qualifiers[f][q]; @@ -267,29 +244,25 @@ protected RowData convertToRow( byte[] value = result.getValue(familyKey, qualifier); familyRow.setField(q, qualifierDecoders[f][q].decode(value)); } - resultRow.setField(i, familyRow); + reusedRow.setField(i, familyRow); } } - return resultRow; + return reusedRow; } - /** - * Converts HBase {@link Result} into {@link RowData}. - * - * @deprecated Use {@link #convertToReusedRow(Result)} instead. - */ - @Deprecated - public RowData convertToRow(Result result) { + /** Converts HBase {@link Result} into {@link RowData}. */ + public RowData convertToNewRow(Result result) { + GenericRowData rowData = new GenericRowData(fieldLength); for (int i = 0; i < fieldLength; i++) { if (rowkeyIndex == i) { assert keyDecoder != null; Object rowkey = keyDecoder.decode(result.getRow()); - reusedRow.setField(rowkeyIndex, rowkey); + rowData.setField(rowkeyIndex, rowkey); } else { int f = (rowkeyIndex != -1 && i > rowkeyIndex) ? i - 1 : i; // get family key byte[] familyKey = families[f]; - GenericRowData familyRow = reusedFamilyRows[f]; + GenericRowData familyRow = new GenericRowData(this.qualifiers[f].length); for (int q = 0; q < this.qualifiers[f].length; q++) { // get quantifier key byte[] qualifier = qualifiers[f][q]; @@ -297,10 +270,10 @@ public RowData convertToRow(Result result) { byte[] value = result.getValue(familyKey, qualifier); familyRow.setField(q, qualifierDecoders[f][q].decode(value)); } - reusedRow.setField(i, familyRow); + rowData.setField(i, familyRow); } } - return reusedRow; + return rowData; } // ------------------------------------------------------------------------------------ @@ -309,11 +282,11 @@ public RowData convertToRow(Result result) { /** Runtime encoder that encodes a specified field in {@link RowData} into byte[]. */ @FunctionalInterface - protected interface FieldEncoder extends Serializable { + public interface FieldEncoder extends Serializable { byte[] encode(RowData row, int pos); } - protected static FieldEncoder createNullableFieldEncoder( + private static FieldEncoder createNullableFieldEncoder( LogicalType fieldType, final byte[] nullStringBytes) { final FieldEncoder encoder = createFieldEncoder(fieldType); if (fieldType.isNullable()) { @@ -342,7 +315,7 @@ protected static FieldEncoder createNullableFieldEncoder( } } - protected static FieldEncoder createFieldEncoder(LogicalType fieldType) { + private static FieldEncoder createFieldEncoder(LogicalType fieldType) { // ordered by type root definition switch (fieldType.getTypeRoot()) { case CHAR: @@ -400,7 +373,7 @@ protected static FieldEncoder createFieldEncoder(LogicalType fieldType) { } } - protected static FieldEncoder createDecimalEncoder(DecimalType decimalType) { + private static FieldEncoder createDecimalEncoder(DecimalType decimalType) { final int precision = decimalType.getPrecision(); final int scale = decimalType.getScale(); return (row, pos) -> { @@ -409,7 +382,7 @@ protected static FieldEncoder createDecimalEncoder(DecimalType decimalType) { }; } - protected static FieldEncoder createTimestampEncoder(final int precision) { + private static FieldEncoder createTimestampEncoder(final int precision) { return (row, pos) -> { long millisecond = row.getTimestamp(pos, precision).getMillisecond(); return Bytes.toBytes(millisecond); @@ -427,7 +400,7 @@ protected interface FieldDecoder extends Serializable { Object decode(byte[] value); } - protected static FieldDecoder createNullableFieldDecoder( + private static FieldDecoder createNullableFieldDecoder( LogicalType fieldType, final byte[] nullStringBytes) { final FieldDecoder decoder = createFieldDecoder(fieldType); if (fieldType.isNullable()) { @@ -453,7 +426,7 @@ protected static FieldDecoder createNullableFieldDecoder( } } - protected static FieldDecoder createFieldDecoder(LogicalType fieldType) { + private static FieldDecoder createFieldDecoder(LogicalType fieldType) { // ordered by type root definition switch (fieldType.getTypeRoot()) { case CHAR: @@ -511,7 +484,7 @@ protected static FieldDecoder createFieldDecoder(LogicalType fieldType) { } } - protected static FieldDecoder createDecimalDecoder(DecimalType decimalType) { + private static FieldDecoder createDecimalDecoder(DecimalType decimalType) { final int precision = decimalType.getPrecision(); final int scale = decimalType.getScale(); return value -> { @@ -520,7 +493,7 @@ protected static FieldDecoder createDecimalDecoder(DecimalType decimalType) { }; } - protected static FieldDecoder createTimestampDecoder() { + private static FieldDecoder createTimestampDecoder() { return value -> { // TODO: support higher precision long milliseconds = Bytes.toLong(value); @@ -528,6 +501,16 @@ protected static FieldDecoder createTimestampDecoder() { }; } + @Nullable + public FieldEncoder getKeyEncoder() { + return keyEncoder; + } + + @Nullable + public FieldDecoder getKeyDecoder() { + return keyDecoder; + } + public byte[] getRowKey(Object rowKey) { checkArgument(keyEncoder != null, "row key is not set."); rowWithRowKey.setField(0, rowKey); diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HbaseRowConverter.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HbaseRowConverter.java new file mode 100644 index 0000000000..7379964795 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HbaseRowConverter.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.hbase14.converter; + +import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; +import com.dtstack.chunjun.converter.AbstractRowConverter; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; + +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Result; + +public class HbaseRowConverter + extends AbstractRowConverter { + private HBaseTableSchema schema; + private String nullStringLiteral; + private transient HBaseSerde serde; + + public HbaseRowConverter(HBaseTableSchema schema, String nullStringLiteral) { + // super(rowType); + this.schema = schema; + this.nullStringLiteral = nullStringLiteral; + } + + @Override + public RowData toInternal(Result input) throws Exception { + if (serde == null) { + this.serde = new HBaseSerde(schema, nullStringLiteral); + } + + return serde.convertToRow(input); + } + + @Override + public Mutation toExternal(RowData rowData, Mutation output) throws Exception { + if (serde == null) { + this.serde = new HBaseSerde(schema, nullStringLiteral); + } + RowKind kind = rowData.getRowKind(); + if (kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER) { + return serde.createPutMutation(rowData); + } else { + return serde.createDeleteMutation(rowData); + } + } + + @Override + public RowData toInternalLookup(RowData input) throws Exception { + return input; + } +} diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBase14SinkFactory.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBase14SinkFactory.java index 00db12d245..f73624d372 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBase14SinkFactory.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBase14SinkFactory.java @@ -16,17 +16,28 @@ package com.dtstack.chunjun.connector.hbase14.sink; import com.dtstack.chunjun.conf.SyncConf; -import com.dtstack.chunjun.connector.hbase.HBaseColumnConverter; -import com.dtstack.chunjun.connector.hbase14.conf.HBaseConf; -import com.dtstack.chunjun.connector.hbase14.converter.HBaseRawTypeConverter; +import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.converter.HBaseRawTypeConverter; +import com.dtstack.chunjun.connector.hbase14.converter.HBaseColumnConverter; +import com.dtstack.chunjun.connector.hbase14.converter.HbaseRowConverter; 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 com.dtstack.chunjun.util.TableUtil; +import com.dtstack.chunjun.util.ValueUtil; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.commons.lang.StringUtils; + +import java.util.List; +import java.util.Map; public class HBase14SinkFactory extends SinkFactory { @@ -38,32 +49,91 @@ public HBase14SinkFactory(SyncConf config) { GsonUtil.GSON.fromJson( GsonUtil.GSON.toJson(config.getWriter().getParameter()), HBaseConf.class); super.initCommonConf(hbaseConf); - hbaseConf.setColumnMetaInfos(syncConf.getWriter().getFieldList()); + hbaseConf.setColumn(syncConf.getWriter().getFieldList()); + + if (config.getWriter().getParameter().get("rowkeyColumn") != null) { + String rowkeyColumn = + buildRowKeyExpress(config.getWriter().getParameter().get("rowkeyColumn")); + hbaseConf.setRowkeyExpress(rowkeyColumn); + } + + if (config.getWriter().getParameter().get("versionColumn") != null) { + Map versionColumn = + (Map) config.getWriter().getParameter().get("versionColumn"); + if (null != versionColumn.get("index") + && StringUtils.isNotBlank(versionColumn.get("index").toString())) { + hbaseConf.setVersionColumnIndex( + Integer.valueOf(versionColumn.get("index").toString())); + } + + if (null != versionColumn.get("value") + && StringUtils.isNotBlank(versionColumn.get("value").toString())) { + hbaseConf.setVersionColumnValue(versionColumn.get("value").toString()); + } + } } @Override public DataStreamSink createSink(DataStream dataSet) { HBaseOutputFormatBuilder builder = new HBaseOutputFormatBuilder(); builder.setConfig(hbaseConf); - builder.setColumnMetaInfos(hbaseConf.getColumnMetaInfos()); - builder.setEncoding(hbaseConf.getEncoding()); + builder.setHbaseConf(hbaseConf); + builder.setHbaseConfig(hbaseConf.getHbaseConfig()); - builder.setNullMode(hbaseConf.getNullMode()); - builder.setRowkeyExpress(hbaseConf.getRowkeyExpress()); builder.setTableName(hbaseConf.getTable()); - builder.setVersionColumnIndex(hbaseConf.getVersionColumnIndex()); - builder.setVersionColumnValues(hbaseConf.getVersionColumnValue()); - builder.setWalFlag(hbaseConf.getWalFlag()); - builder.setRowkeyExpress(hbaseConf.getRowkeyExpress()); builder.setWriteBufferSize(hbaseConf.getWriteBufferSize()); - AbstractRowConverter rowConverter = - new HBaseColumnConverter(hbaseConf.getColumnMetaInfos()); + AbstractRowConverter rowConverter; + if (useAbstractBaseColumn) { + final RowType rowType = + TableUtil.createRowType(hbaseConf.getColumn(), getRawTypeConverter()); + rowConverter = new HBaseColumnConverter(hbaseConf, rowType); + } else { + TableSchema tableSchema = + TableUtil.createTableSchema(hbaseConf.getColumn(), getRawTypeConverter()); + HBaseTableSchema hbaseSchema = HBaseTableSchema.fromTableSchema(tableSchema); + String nullStringLiteral = hbaseConf.getNullStringLiteral(); + rowConverter = new HbaseRowConverter(hbaseSchema, nullStringLiteral); + } + builder.setRowConverter(rowConverter); return createOutput(dataSet, builder.finish()); } @Override public RawTypeConverter getRawTypeConverter() { - return new HBaseRawTypeConverter(); + return HBaseRawTypeConverter::apply; + } + + /** Compatible with old formats */ + private String buildRowKeyExpress(Object rowKeyInfo) { + if (rowKeyInfo == null) { + return null; + } + + if (rowKeyInfo instanceof String) { + return rowKeyInfo.toString(); + } + + if (!(rowKeyInfo instanceof List)) { + return null; + } + + StringBuilder expressBuilder = new StringBuilder(); + + for (Map item : ((List) rowKeyInfo)) { + Integer index = ValueUtil.getInt(item.get("index")); + if (index != null && index != -1) { + expressBuilder.append( + String.format("$(%s)", hbaseConf.getColumn().get(index).getName())); + continue; + } + + String value = (String) item.get("value"); + if (StringUtils.isNotEmpty(value)) { + expressBuilder.append(value); + } + } + + return expressBuilder.toString(); } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBaseOutputFormat.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBaseOutputFormat.java index 8391873e37..12e362dddd 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBaseOutputFormat.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBaseOutputFormat.java @@ -18,9 +18,8 @@ package com.dtstack.chunjun.connector.hbase14.sink; -import com.dtstack.chunjun.connector.hbase.HBaseMutationConverter; -import com.dtstack.chunjun.connector.hbase14.converter.DataSyncSinkConverter; -import com.dtstack.chunjun.connector.hbase14.util.HBaseConfigUtils; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.util.HBaseConfigUtils; import com.dtstack.chunjun.connector.hbase14.util.HBaseHelper; import com.dtstack.chunjun.sink.format.BaseRichOutputFormat; import com.dtstack.chunjun.throwable.WriteRecordException; @@ -58,31 +57,13 @@ public class HBaseOutputFormat extends BaseRichOutputFormat { private Map hbaseConfig; private String tableName; - private String encoding; - private String nullMode; - private boolean walFlag; private long writeBufferSize; - private List columnTypes; - private List columnNames; - - private String rowkeyExpress; - private Integer versionColumnIndex; - - private String versionColumnValue; - private transient Connection connection; private transient BufferedMutator bufferedMutator; private transient Table table; - private HBaseMutationConverter mutationConverter; - private DataSyncSinkConverter dataSyncSinkConverter; - - public void setMutationConverter(HBaseMutationConverter mutationConverter) { - this.mutationConverter = mutationConverter; - } - @Override public void configure(Configuration parameters) {} @@ -90,11 +71,9 @@ public void configure(Configuration parameters) {} protected void writeSingleRecordInternal(RowData rawRecord) throws WriteRecordException { int i = 0; try { - if (mutationConverter != null) { - bufferedMutator.mutate(mutationConverter.convertToMutation(rawRecord)); - } else { - bufferedMutator.mutate(dataSyncSinkConverter.generatePutCommand(rawRecord)); - } + + bufferedMutator.mutate((Mutation) rowConverter.toExternal(rawRecord, null)); + } catch (Exception ex) { if (i < rawRecord.getArity()) { throw new WriteRecordException( @@ -118,20 +97,6 @@ public void openInternal(int taskNumber, int numTasks) throws IOException { } else { openConnection(); } - if (mutationConverter != null) { - mutationConverter.open(); - } else { - dataSyncSinkConverter = - new DataSyncSinkConverter( - walFlag, - nullMode, - encoding, - columnTypes, - columnNames, - rowkeyExpress, - versionColumnIndex, - versionColumnValue); - } } public void openConnection() { @@ -164,11 +129,7 @@ protected void writeMultipleRecordsInternal() throws Exception { try { List mutations = new ArrayList<>(); for (RowData record : rows) { - if (mutationConverter != null) { - mutations.add(mutationConverter.convertToMutation(record)); - } else { - mutations.add(dataSyncSinkConverter.generatePutCommand(record)); - } + mutations.add((Mutation) rowConverter.toExternal(record, null)); } results = new Object[mutations.size()]; table.batch(mutations, results); @@ -186,9 +147,7 @@ protected void writeMultipleRecordsInternal() throws Exception { @Override public void closeInternal() throws IOException { - if (dataSyncSinkConverter != null) { - dataSyncSinkConverter.close(); - } + HBaseHelper.closeBufferedMutator(bufferedMutator); HBaseHelper.closeConnection(connection); } @@ -197,59 +156,23 @@ public void setTableName(String tableName) { this.tableName = tableName; } - public void setHbaseConfig(Map hbaseConfig) { + public void setHbaseConf(Map hbaseConfig) { this.hbaseConfig = hbaseConfig; } - public void setColumnTypes(List columnTypes) { - this.columnTypes = columnTypes; - } - - public void setColumnNames(List columnNames) { - this.columnNames = columnNames; - } - - public void setRowkeyExpress(String rowkeyExpress) { - this.rowkeyExpress = rowkeyExpress; - } - - public void setVersionColumnIndex(Integer versionColumnIndex) { - this.versionColumnIndex = versionColumnIndex; - } - - public void setVersionColumnValue(String versionColumnValue) { - this.versionColumnValue = versionColumnValue; - } - - public void setEncoding(String defaultEncoding) { - this.encoding = defaultEncoding; - } - public void setWriteBufferSize(Long writeBufferSize) { this.writeBufferSize = writeBufferSize; } - public void setNullMode(String nullMode) { - this.nullMode = nullMode; - } - - public void setWalFlag(Boolean walFlag) { - this.walFlag = walFlag; - } - public String getTableName() { return tableName; } - public List getColumnNames() { - return columnNames; - } - - public List getColumnTypes() { - return columnTypes; - } - public Map getHbaseConfig() { return hbaseConfig; } + + public void setHbaseConf(HBaseConf config) { + this.config = config; + } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBaseOutputFormatBuilder.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBaseOutputFormatBuilder.java index e2f5aae790..b1613324d7 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBaseOutputFormatBuilder.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/HBaseOutputFormatBuilder.java @@ -18,17 +18,14 @@ package com.dtstack.chunjun.connector.hbase14.sink; -import com.dtstack.chunjun.conf.FieldConf; -import com.dtstack.chunjun.connector.hbase.HBaseMutationConverter; -import com.dtstack.chunjun.connector.hbase14.conf.HBaseConfigConstants; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConfigConstants; import com.dtstack.chunjun.sink.format.BaseRichOutputFormatBuilder; import com.google.common.base.Preconditions; import org.apache.commons.lang.StringUtils; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** * The Builder class of HbaseOutputFormatBuilder @@ -45,44 +42,16 @@ public HBaseOutputFormatBuilder() { super.format = format = new HBaseOutputFormat(); } + public void setHbaseConf(HBaseConf config) { + format.setHbaseConf(config); + } + public void setTableName(String tableName) { format.setTableName(tableName); } public void setHbaseConfig(Map hbaseConfig) { - format.setHbaseConfig(hbaseConfig); - } - - public void setColumnTypes(List columnTypes) { - format.setColumnTypes(columnTypes); - } - - public void setColumnNames(List columnNames) { - format.setColumnNames(columnNames); - } - - public void setRowkeyExpress(String rowkeyExpress) { - format.setRowkeyExpress(rowkeyExpress); - } - - public void setVersionColumnIndex(Integer versionColumnIndex) { - format.setVersionColumnIndex(versionColumnIndex); - } - - public void setVersionColumnValues(String versionColumnValue) { - format.setVersionColumnValue(versionColumnValue); - } - - public void setHBaseMutationConverter(HBaseMutationConverter hbaseMutationConverter) { - format.setMutationConverter(hbaseMutationConverter); - } - - public void setEncoding(String encoding) { - if (StringUtils.isEmpty(encoding)) { - format.setEncoding(HBaseConfigConstants.DEFAULT_ENCODING); - } else { - format.setEncoding(encoding); - } + format.setHbaseConf(hbaseConfig); } public void setWriteBufferSize(Long writeBufferSize) { @@ -93,38 +62,9 @@ public void setWriteBufferSize(Long writeBufferSize) { } } - public void setNullMode(String nullMode) { - if (StringUtils.isEmpty(nullMode)) { - format.setNullMode(HBaseConfigConstants.DEFAULT_NULL_MODE); - } else { - format.setNullMode(nullMode); - } - } - - public void setWalFlag(Boolean walFlag) { - if (walFlag == null) { - format.setWalFlag(false); - } else { - format.setWalFlag(walFlag); - } - } - @Override protected void checkFormat() { Preconditions.checkArgument(StringUtils.isNotEmpty(format.getTableName())); Preconditions.checkNotNull(format.getHbaseConfig()); - Preconditions.checkNotNull(format.getColumnNames()); - Preconditions.checkNotNull(format.getColumnTypes()); - } - - public void setColumnMetaInfos(List columnMetaInfos) { - if (columnMetaInfos != null && !columnMetaInfos.isEmpty()) { - List names = - columnMetaInfos.stream().map(FieldConf::getName).collect(Collectors.toList()); - setColumnNames(names); - List values = - columnMetaInfos.stream().map(FieldConf::getType).collect(Collectors.toList()); - setColumnTypes(values); - } } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBase14SourceFactory.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBase14SourceFactory.java index b1a63be231..62884de568 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBase14SourceFactory.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBase14SourceFactory.java @@ -16,21 +16,29 @@ package com.dtstack.chunjun.connector.hbase14.source; import com.dtstack.chunjun.conf.SyncConf; -import com.dtstack.chunjun.connector.hbase.HBaseColumnConverter; -import com.dtstack.chunjun.connector.hbase14.conf.HBaseConf; -import com.dtstack.chunjun.connector.hbase14.converter.HBaseRawTypeConverter; +import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.converter.HBaseRawTypeConverter; +import com.dtstack.chunjun.connector.hbase14.converter.HBaseColumnConverter; +import com.dtstack.chunjun.connector.hbase14.converter.HbaseRowConverter; 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 com.dtstack.chunjun.util.TableUtil; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; + public class HBase14SourceFactory extends SourceFactory { private static final Logger LOG = LoggerFactory.getLogger(HBase14SourceFactory.class); @@ -42,30 +50,53 @@ public HBase14SourceFactory(SyncConf syncConf, StreamExecutionEnvironment env) { config = GsonUtil.GSON.fromJson( GsonUtil.GSON.toJson(syncConf.getReader().getParameter()), HBaseConf.class); + Map range = + (Map) syncConf.getReader().getParameter().get("range"); + if (range != null) { + if (range.get("startRowkey") != null + && StringUtils.isNotBlank(range.get("startRowkey").toString())) { + config.setStartRowkey(range.get("startRowkey").toString()); + } + if (range.get("endRowkey") != null + && StringUtils.isNotBlank(range.get("endRowkey").toString())) { + config.setEndRowkey(range.get("endRowkey").toString()); + } + if (range.get("isBinaryRowkey") != null) { + config.setBinaryRowkey((Boolean) range.get("isBinaryRowkey")); + } + } + super.initCommonConf(config); - config.setColumnMetaInfos(syncConf.getReader().getFieldList()); + config.setColumn(syncConf.getReader().getFieldList()); } @Override public RawTypeConverter getRawTypeConverter() { - return new HBaseRawTypeConverter(); + return HBaseRawTypeConverter::apply; } @Override @SuppressWarnings("all") public DataStream createSource() { HBaseInputFormatBuilder builder = new HBaseInputFormatBuilder(); - builder.setColumnMetaInfos(config.getColumnMetaInfos()); builder.setConfig(config); - builder.setColumnMetaInfos(config.getColumnMetaInfos()); - builder.setEncoding(config.getEncoding()); + builder.sethHBaseConf(config); + builder.setHbaseConfig(config.getHbaseConfig()); - builder.setTableName(config.getTable()); - builder.setEndRowKey(config.getEndRowkey()); - builder.setIsBinaryRowkey(config.isBinaryRowkey()); - builder.setScanCacheSize(config.getScanCacheSize()); - builder.setStartRowKey(config.getStartRowkey()); - AbstractRowConverter rowConverter = new HBaseColumnConverter(config.getColumnMetaInfos()); + + AbstractRowConverter rowConverter; + if (useAbstractBaseColumn) { + final RowType rowType = + TableUtil.createRowType(config.getColumn(), getRawTypeConverter()); + rowConverter = new HBaseColumnConverter(config, rowType); + } else { + TableSchema tableSchema = + TableUtil.createTableSchema(config.getColumn(), getRawTypeConverter()); + HBaseTableSchema hbaseSchema = HBaseTableSchema.fromTableSchema(tableSchema); + String nullStringLiteral = config.getNullStringLiteral(); + rowConverter = new HbaseRowConverter(hbaseSchema, nullStringLiteral); + } + builder.setRowConverter(rowConverter); return createInput(builder.finish()); } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBaseInputFormat.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBaseInputFormat.java index 28a8852cb1..45cf28d314 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBaseInputFormat.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBaseInputFormat.java @@ -18,19 +18,15 @@ package com.dtstack.chunjun.connector.hbase14.source; -import com.dtstack.chunjun.connector.hbase14.util.HBaseConfigUtils; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.util.HBaseConfigUtils; import com.dtstack.chunjun.connector.hbase14.util.HBaseHelper; import com.dtstack.chunjun.source.format.BaseRichInputFormat; +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.collect.Maps; -import org.apache.commons.lang.ArrayUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.commons.lang3.math.NumberUtils; -import org.apache.commons.lang3.time.DateUtils; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; @@ -44,12 +40,9 @@ import org.apache.hadoop.security.UserGroupInformation; import java.io.IOException; -import java.math.BigDecimal; -import java.nio.charset.StandardCharsets; import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.List; -import java.util.Locale; import java.util.Map; /** @@ -61,36 +54,20 @@ */ public class HBaseInputFormat extends BaseRichInputFormat { - public static final String KEY_ROW_KEY = "rowkey"; - protected Map hbaseConfig; - protected String tableName; - protected String startRowkey; - protected String endRowkey; - protected List columnNames; - protected List columnValues; - protected List columnFormats; - protected List columnTypes; - protected boolean isBinaryRowkey; - protected String encoding; - /** 客户端每次 rpc fetch 的行数 */ - protected int scanCacheSize = 1000; + protected HBaseConf hBaseConf; private transient Connection connection; private transient Scan scan; private transient Table table; private transient ResultScanner resultScanner; private transient Result next; - private transient Map nameMaps; - - private boolean openKerberos = false; @Override public void openInputFormat() throws IOException { super.openInputFormat(); LOG.info("HbaseOutputFormat openInputFormat start"); - nameMaps = Maps.newConcurrentMap(); connection = HBaseHelper.getHbaseConnection(hbaseConfig); @@ -107,16 +84,60 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOExcepti () -> split( connection, - tableName, - startRowkey, - endRowkey, - isBinaryRowkey)); + hBaseConf.getTable(), + hBaseConf.getStartRowkey(), + hBaseConf.getEndRowkey(), + hBaseConf.isBinaryRowkey())); } else { - return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + return split( + connection, + hBaseConf.getTable(), + hBaseConf.getStartRowkey(), + hBaseConf.getEndRowkey(), + hBaseConf.isBinaryRowkey()); } } } + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + HBaseInputSplit hbaseInputSplit = (HBaseInputSplit) inputSplit; + byte[] startRow = Bytes.toBytesBinary(hbaseInputSplit.getStartkey()); + byte[] stopRow = Bytes.toBytesBinary(hbaseInputSplit.getEndKey()); + + if (null == connection || connection.isClosed()) { + connection = HBaseHelper.getHbaseConnection(hbaseConfig); + } + + table = connection.getTable(TableName.valueOf(hBaseConf.getTable())); + scan = new Scan(); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.setCaching(hBaseConf.getScanCacheSize()); + resultScanner = table.getScanner(scan); + } + + @Override + public boolean reachedEnd() throws IOException { + next = resultScanner.next(); + return next == null; + } + + @Override + public RowData nextRecordInternal(RowData rawRow) throws ReadRecordException { + try { + rawRow = rowConverter.toInternal(next); + return rawRow; + } catch (Exception se) { + throw new ReadRecordException("", se, 0, rawRow); + } + } + + @Override + public void closeInternal() throws IOException { + HBaseHelper.closeConnection(connection); + } + public HBaseInputSplit[] split( Connection hConn, String tableName, @@ -234,150 +255,4 @@ private String getStartKey(byte[] startRowkeyByte, byte[] regionStarKey) { } return Bytes.toStringBinary(tempStartRowkeyByte); } - - @Override - public void openInternal(InputSplit inputSplit) throws IOException { - HBaseInputSplit hbaseInputSplit = (HBaseInputSplit) inputSplit; - byte[] startRow = Bytes.toBytesBinary(hbaseInputSplit.getStartkey()); - byte[] stopRow = Bytes.toBytesBinary(hbaseInputSplit.getEndKey()); - - if (null == connection || connection.isClosed()) { - connection = HBaseHelper.getHbaseConnection(hbaseConfig); - } - - openKerberos = HBaseConfigUtils.isEnableKerberos(hbaseConfig); - - table = connection.getTable(TableName.valueOf(tableName)); - scan = new Scan(); - scan.setStartRow(startRow); - scan.setStopRow(stopRow); - scan.setCaching(scanCacheSize); - resultScanner = table.getScanner(scan); - } - - @Override - public boolean reachedEnd() throws IOException { - next = resultScanner.next(); - return next == null; - } - - @Override - public RowData nextRecordInternal(RowData rawRow) { - GenericRowData row = new GenericRowData(columnTypes.size()); - - for (int i = 0; i < columnTypes.size(); ++i) { - String columnType = columnTypes.get(i); - String columnName = columnNames.get(i); - String columnFormat = columnFormats.get(i); - String columnValue = columnValues.get(i); - Object col = null; - byte[] bytes; - - try { - if (StringUtils.isNotEmpty(columnValue)) { - // 常量 - col = convertValueToAssignType(columnType, columnValue, columnFormat); - } else { - if (KEY_ROW_KEY.equals(columnName)) { - bytes = next.getRow(); - } else { - byte[][] arr = nameMaps.get(columnName); - if (arr == null) { - arr = new byte[2][]; - String[] arr1 = columnName.split(":"); - arr[0] = arr1[0].trim().getBytes(StandardCharsets.UTF_8); - arr[1] = arr1[1].trim().getBytes(StandardCharsets.UTF_8); - nameMaps.put(columnName, arr); - } - bytes = next.getValue(arr[0], arr[1]); - } - col = convertBytesToAssignType(columnType, bytes, columnFormat); - } - row.setField(i, col); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - return row; - } - - @Override - public void closeInternal() throws IOException { - HBaseHelper.closeConnection(connection); - } - - public Object convertValueToAssignType( - String columnType, String constantValue, String dateformat) throws Exception { - Object column = null; - if (org.apache.commons.lang3.StringUtils.isEmpty(constantValue)) { - return column; - } - - switch (columnType.toUpperCase()) { - case "BOOLEAN": - column = Boolean.valueOf(constantValue); - break; - case "SHORT": - case "INT": - case "LONG": - column = NumberUtils.createBigDecimal(constantValue).toBigInteger(); - break; - case "FLOAT": - case "DOUBLE": - column = new BigDecimal(constantValue); - break; - case "STRING": - column = constantValue; - break; - case "DATE": - column = DateUtils.parseDate(constantValue, new String[] {dateformat}); - break; - default: - throw new IllegalArgumentException("Unsupported columnType: " + columnType); - } - - return column; - } - - public Object convertBytesToAssignType(String columnType, byte[] byteArray, String dateformat) - throws Exception { - Object column = null; - if (ArrayUtils.isEmpty(byteArray)) { - return null; - } - String bytesToString = new String(byteArray, encoding); - switch (columnType.toUpperCase(Locale.ENGLISH)) { - case "BOOLEAN": - column = Boolean.valueOf(bytesToString); - break; - case "SHORT": - column = Short.valueOf(bytesToString); - break; - case "INT": - column = Integer.valueOf(bytesToString); - break; - case "LONG": - column = Long.valueOf(bytesToString); - break; - case "FLOAT": - column = Float.valueOf(bytesToString); - break; - case "DOUBLE": - column = Double.valueOf(bytesToString); - break; - case "STRING": - column = bytesToString; - break; - case "BINARY_STRING": - column = Bytes.toStringBinary(byteArray); - break; - case "DATE": - String dateValue = Bytes.toStringBinary(byteArray); - column = DateUtils.parseDate(dateValue, new String[] {dateformat}); - break; - default: - throw new IllegalArgumentException("Unsupported column type: " + columnType); - } - return column; - } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBaseInputFormatBuilder.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBaseInputFormatBuilder.java index dd4dd94744..5ae57377ef 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBaseInputFormatBuilder.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/source/HBaseInputFormatBuilder.java @@ -17,17 +17,13 @@ */ package com.dtstack.chunjun.connector.hbase14.source; -import com.dtstack.chunjun.conf.FieldConf; -import com.dtstack.chunjun.connector.hbase14.conf.HBaseConfigConstants; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConfigConstants; import com.dtstack.chunjun.source.format.BaseRichInputFormatBuilder; import org.apache.flink.util.Preconditions; -import org.apache.commons.lang.StringUtils; - -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** * The builder of HbaseInputFormat @@ -36,90 +32,30 @@ * * @author huyifan.zju@163.com */ -public class HBaseInputFormatBuilder extends BaseRichInputFormatBuilder { +public class HBaseInputFormatBuilder extends BaseRichInputFormatBuilder { public HBaseInputFormatBuilder() { super(new HBaseInputFormat()); } public void setHbaseConfig(Map hbaseConfig) { - format.hbaseConfig = hbaseConfig; - } - - public void setTableName(String tableName) { - format.tableName = tableName; - } - - public void setStartRowKey(String startRowKey) { - format.startRowkey = startRowKey; - } - - public void setEndRowKey(String endRowKey) { - format.endRowkey = endRowKey; - } - - public void setColumnNames(List columnNames) { - format.columnNames = columnNames; - } - - public void setColumnValues(List columnValues) { - format.columnValues = columnValues; + ((HBaseInputFormat) format).hbaseConfig = hbaseConfig; } - public void setColumnTypes(List columnTypes) { - format.columnTypes = columnTypes; - } - - public void setIsBinaryRowkey(boolean isBinaryRowkey) { - format.isBinaryRowkey = isBinaryRowkey; - } - - public void setEncoding(String encoding) { - format.encoding = StringUtils.isEmpty(encoding) ? "utf-8" : encoding; - } - - public void setColumnFormats(List columnFormats) { - format.columnFormats = columnFormats; - } - - public void setScanCacheSize(int scanCacheSize) { - format.scanCacheSize = scanCacheSize; + public void sethHBaseConf(HBaseConf hBaseConf) { + ((HBaseInputFormat) format).hBaseConf = hBaseConf; } @Override protected void checkFormat() { Preconditions.checkArgument( - format.scanCacheSize <= HBaseConfigConstants.MAX_SCAN_CACHE_SIZE - && format.scanCacheSize >= HBaseConfigConstants.MIN_SCAN_CACHE_SIZE, + ((HBaseInputFormat) format).hBaseConf.getScanCacheSize() + <= HBaseConfigConstants.MAX_SCAN_CACHE_SIZE + && ((HBaseInputFormat) format).hBaseConf.getScanCacheSize() + >= HBaseConfigConstants.MIN_SCAN_CACHE_SIZE, "scanCacheSize should be between " + HBaseConfigConstants.MIN_SCAN_CACHE_SIZE + " and " + HBaseConfigConstants.MAX_SCAN_CACHE_SIZE); - - if (format.columnFormats != null) { - for (int i = 0; i < format.columnTypes.size(); ++i) { - Preconditions.checkArgument(StringUtils.isNotEmpty(format.columnTypes.get(i))); - Preconditions.checkArgument( - StringUtils.isNotEmpty(format.columnNames.get(i)) - || StringUtils.isNotEmpty(format.columnTypes.get(i))); - } - } - } - - public void setColumnMetaInfos(List columMetaInfos) { - if (columMetaInfos != null && !columMetaInfos.isEmpty()) { - List nameList = - columMetaInfos.stream().map(FieldConf::getName).collect(Collectors.toList()); - setColumnNames(nameList); - List typeList = - columMetaInfos.stream().map(FieldConf::getType).collect(Collectors.toList()); - setColumnTypes(typeList); - List valueList = - columMetaInfos.stream().map(FieldConf::getValue).collect(Collectors.toList()); - setColumnValues(valueList); - List formatList = - columMetaInfos.stream().map(FieldConf::getFormat).collect(Collectors.toList()); - setColumnFormats(formatList); - } } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/HBaseDynamicTableSink.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/HBaseDynamicTableSink.java index 29dfffafa2..fa71d321fc 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/HBaseDynamicTableSink.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/HBaseDynamicTableSink.java @@ -17,12 +17,9 @@ */ package com.dtstack.chunjun.connector.hbase14.table; -import com.dtstack.chunjun.conf.FieldConf; -import com.dtstack.chunjun.connector.hbase.HBaseConverter; -import com.dtstack.chunjun.connector.hbase.HBaseMutationConverter; import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; -import com.dtstack.chunjun.connector.hbase.RowDataToMutationConverter; -import com.dtstack.chunjun.connector.hbase14.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase14.converter.HbaseRowConverter; import com.dtstack.chunjun.connector.hbase14.sink.HBaseOutputFormatBuilder; import com.dtstack.chunjun.converter.AbstractRowConverter; import com.dtstack.chunjun.sink.DtOutputFormatSinkFunction; @@ -31,7 +28,10 @@ import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; import java.util.ArrayList; import java.util.List; @@ -61,41 +61,36 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { @Override public SinkFunctionProvider getSinkRuntimeProvider(Context context) { - final RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); - String[] fieldNames = tableSchema.getFieldNames(); - List columnList = new ArrayList<>(fieldNames.length); - for (int i = 0; i < fieldNames.length; i++) { - FieldConf field = new FieldConf(); - field.setName(fieldNames[i]); - field.setType(rowType.getTypeAt(i).asSummaryString()); - field.setIndex(i); - columnList.add(field); + List logicalTypes = new ArrayList<>(); + + String[] familyNames = hbaseSchema.getFamilyNames(); + int rowKeyIndex = hbaseSchema.getRowKeyIndex(); + for (int i = 0; i < familyNames.length; i++) { + if (i == rowKeyIndex) { + logicalTypes.add( + TypeConversions.fromDataToLogicalType( + hbaseSchema.getRowKeyDataType().get())); + } + DataType[] qualifierDataTypes = hbaseSchema.getQualifierDataTypes(familyNames[i]); + for (DataType dataType : qualifierDataTypes) { + logicalTypes.add(TypeConversions.fromDataToLogicalType(dataType)); + } } + // todo 测试下顺序是否是一致的 + RowType of = RowType.of(logicalTypes.toArray(new LogicalType[0])); + HBaseOutputFormatBuilder builder = new HBaseOutputFormatBuilder(); - if (conf.getColumn() != null) { - builder.setColumnMetaInfos(conf.getColumn()); - } else if (conf.getColumnMetaInfos() != null) { - builder.setColumnMetaInfos(conf.getColumnMetaInfos()); - } else if (!columnList.isEmpty()) { - builder.setColumnMetaInfos(columnList); - } - builder.setEncoding(conf.getEncoding()); + builder.setConfig(conf); + builder.setHbaseConf(conf); builder.setHbaseConfig(conf.getHbaseConfig()); - builder.setNullMode(conf.getNullMode()); - builder.setTableName(conf.getTableName()); - builder.setRowkeyExpress(conf.getRowkeyExpress()); - builder.setVersionColumnIndex(conf.getVersionColumnIndex()); - builder.setVersionColumnValues(conf.getVersionColumnValue()); - builder.setWalFlag(conf.getWalFlag()); + builder.setTableName(conf.getTable()); + builder.setWriteBufferSize(conf.getWriteBufferSize()); - AbstractRowConverter rowConverter = new HBaseConverter(rowType); - builder.setRowConverter(rowConverter); - builder.setConfig(conf); + String nullStringLiteral = conf.getNullStringLiteral(); - HBaseMutationConverter converter = - new RowDataToMutationConverter(hbaseSchema, conf.getNullMode()); - builder.setHBaseMutationConverter(converter); + AbstractRowConverter rowConverter = new HbaseRowConverter(hbaseSchema, nullStringLiteral); + builder.setRowConverter(rowConverter); return SinkFunctionProvider.of( new DtOutputFormatSinkFunction(builder.finish()), conf.getParallelism()); diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/HBaseDynamicTableSource.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/HBaseDynamicTableSource.java index 13518dfa7b..95c44a5026 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/HBaseDynamicTableSource.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/HBaseDynamicTableSource.java @@ -17,161 +17,67 @@ */ package com.dtstack.chunjun.connector.hbase14.table; -import com.dtstack.chunjun.conf.FieldConf; -import com.dtstack.chunjun.connector.hbase.HBaseConverter; import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; -import com.dtstack.chunjun.connector.hbase14.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.table.BaseHBaseDynamicTableSource; +import com.dtstack.chunjun.connector.hbase.table.lookup.AbstractHBaseAllTableFunction; +import com.dtstack.chunjun.connector.hbase14.converter.HbaseRowConverter; import com.dtstack.chunjun.connector.hbase14.source.HBaseInputFormatBuilder; import com.dtstack.chunjun.connector.hbase14.table.lookup.HBaseAllTableFunction; import com.dtstack.chunjun.connector.hbase14.table.lookup.HBaseLruTableFunction; -import com.dtstack.chunjun.connector.hbase14.util.HBaseConfigUtils; import com.dtstack.chunjun.converter.AbstractRowConverter; -import com.dtstack.chunjun.enums.CacheType; +import com.dtstack.chunjun.lookup.AbstractLruTableFunction; import com.dtstack.chunjun.lookup.conf.LookupConf; -import com.dtstack.chunjun.source.DtInputFormatSourceFunction; -import com.dtstack.chunjun.table.connector.source.ParallelAsyncTableFunctionProvider; -import com.dtstack.chunjun.table.connector.source.ParallelSourceFunctionProvider; -import com.dtstack.chunjun.table.connector.source.ParallelTableFunctionProvider; +import com.dtstack.chunjun.source.format.BaseRichInputFormatBuilder; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.LookupTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.util.Preconditions; - -import org.apache.hadoop.conf.Configuration; - -import java.util.ArrayList; -import java.util.List; /** * Date: 2021/06/17 Company: www.dtstack.com * * @author tudou */ -public class HBaseDynamicTableSource - implements ScanTableSource, LookupTableSource, SupportsProjectionPushDown { - - private final HBaseConf hBaseConf; - private Configuration conf; - private TableSchema tableSchema; - private final LookupConf lookupConf; - private final HBaseTableSchema hbaseSchema; - protected final String nullStringLiteral; +public class HBaseDynamicTableSource extends BaseHBaseDynamicTableSource { public HBaseDynamicTableSource( HBaseConf conf, TableSchema tableSchema, LookupConf lookupConf, - HBaseTableSchema hbaseSchema, - String nullStringLiteral) { - this.hBaseConf = conf; - this.tableSchema = tableSchema; - this.lookupConf = lookupConf; - this.hbaseSchema = hbaseSchema; - this.nullStringLiteral = nullStringLiteral; - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - final RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); - TypeInformation typeInformation = InternalTypeInfo.of(rowType); - String[] fieldNames = tableSchema.getFieldNames(); - List columnList = new ArrayList<>(fieldNames.length); - for (int i = 0; i < fieldNames.length; i++) { - FieldConf field = new FieldConf(); - field.setName(fieldNames[i]); - field.setType(rowType.getTypeAt(i).asSummaryString()); - field.setIndex(i); - columnList.add(field); - } - hBaseConf.setColumn(columnList); - HBaseInputFormatBuilder builder = new HBaseInputFormatBuilder(); - builder.setColumnMetaInfos(hBaseConf.getColumnMetaInfos()); - builder.setConfig(hBaseConf); - builder.setEncoding(hBaseConf.getEncoding()); - builder.setHbaseConfig(hBaseConf.getHbaseConfig()); - builder.setTableName(hBaseConf.getTableName()); - AbstractRowConverter rowConverter = new HBaseConverter(rowType); - builder.setRowConverter(rowConverter); - return ParallelSourceFunctionProvider.of( - new DtInputFormatSourceFunction<>(builder.finish(), typeInformation), - true, - hBaseConf.getParallelism()); + HBaseTableSchema hbaseSchema) { + super(tableSchema, hbaseSchema, conf, lookupConf); } @Override public DynamicTableSource copy() { - return new HBaseDynamicTableSource( - this.hBaseConf, tableSchema, lookupConf, hbaseSchema, nullStringLiteral); + return new HBaseDynamicTableSource(this.hBaseConf, tableSchema, lookupConf, hbaseSchema); } @Override - public String asSummaryString() { - return "HdfsDynamicTableSource:"; - } + protected BaseRichInputFormatBuilder getBaseRichInputFormatBuilder() { + HBaseInputFormatBuilder builder = new HBaseInputFormatBuilder(); + builder.setConfig(hBaseConf); + builder.setHbaseConfig(hBaseConf.getHbaseConfig()); + builder.sethHBaseConf(hBaseConf); - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); + AbstractRowConverter rowConverter = + new HbaseRowConverter(hbaseSchema, hBaseConf.getNullStringLiteral()); + builder.setRowConverter(rowConverter); + return builder; } @Override - public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { - String[] keyNames = new String[context.getKeys().length]; - for (int i = 0; i < keyNames.length; i++) { - int[] innerKeyArr = context.getKeys()[i]; - Preconditions.checkArgument( - innerKeyArr.length == 1, "redis only support non-nested look up keys"); - keyNames[i] = tableSchema.getFieldNames()[innerKeyArr[0]]; - } - setConf(); - hbaseSchema.setTableName(hBaseConf.getTableName()); - if (lookupConf.getCache().equalsIgnoreCase(CacheType.LRU.toString())) { - return ParallelAsyncTableFunctionProvider.of( - new HBaseLruTableFunction( - conf, lookupConf, hbaseSchema, hBaseConf.getNullMode()), - lookupConf.getParallelism()); - } - return ParallelTableFunctionProvider.of( - new HBaseAllTableFunction(conf, lookupConf, hbaseSchema, nullStringLiteral), - lookupConf.getParallelism()); - } - - private void setConf() { - if (HBaseConfigUtils.isEnableKerberos(hBaseConf.getHbaseConfig())) { - conf = HBaseConfigUtils.getHadoopConfiguration(hBaseConf.getHbaseConfig()); - String principal = HBaseConfigUtils.getPrincipal(hBaseConf.getHbaseConfig()); - HBaseConfigUtils.fillSyncKerberosConfig(conf, hBaseConf.getHbaseConfig()); - String keytab = - HBaseConfigUtils.loadKeyFromConf( - hBaseConf.getHbaseConfig(), HBaseConfigUtils.KEY_KEY_TAB); - String krb5Conf = - HBaseConfigUtils.loadKeyFromConf( - hBaseConf.getHbaseConfig(), - HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF); - conf.set(HBaseConfigUtils.KEY_HBASE_CLIENT_KEYTAB_FILE, keytab); - conf.set(HBaseConfigUtils.KEY_HBASE_CLIENT_KERBEROS_PRINCIPAL, principal); - conf.set(HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF, krb5Conf); - } else { - conf = HBaseConfigUtils.getConfig(hBaseConf.getHbaseConfig()); - } + protected AbstractLruTableFunction getAbstractLruTableFunction() { + return new HBaseLruTableFunction(lookupConf, hbaseSchema, hBaseConf); } @Override - public boolean supportsNestedProjection() { - return false; + protected AbstractHBaseAllTableFunction getAbstractAllTableFunction() { + return new HBaseAllTableFunction(lookupConf, hbaseSchema, hBaseConf); } @Override - public void applyProjection(int[][] projectedFields) { - this.tableSchema = TableSchemaUtils.projectSchema(tableSchema, projectedFields); + public String asSummaryString() { + return "Hbase14DynamicTableSource:"; } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/Hbase14DynamicTableFactory.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/Hbase14DynamicTableFactory.java index b92f55460a..26a2d3ff52 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/Hbase14DynamicTableFactory.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/Hbase14DynamicTableFactory.java @@ -16,7 +16,7 @@ package com.dtstack.chunjun.connector.hbase14.table; import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; -import com.dtstack.chunjun.connector.hbase14.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; import com.dtstack.chunjun.lookup.conf.LookupConf; import org.apache.flink.configuration.ConfigOption; @@ -31,12 +31,10 @@ import org.apache.hadoop.hbase.HConstants; -import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; import static com.dtstack.chunjun.connector.hbase14.table.HBaseOptions.NULL_STRING_LITERAL; import static com.dtstack.chunjun.connector.hbase14.table.HBaseOptions.SINK_BUFFER_FLUSH_INTERVAL; @@ -120,8 +118,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { getLookupConf(config, context.getObjectIdentifier().getObjectName()); HBaseTableSchema hbaseSchema = HBaseTableSchema.fromTableSchema(physicalSchema); String nullStringLiteral = helper.getOptions().get(NULL_STRING_LITERAL); - return new HBaseDynamicTableSource( - conf, physicalSchema, lookupConf, hbaseSchema, nullStringLiteral); + return new HBaseDynamicTableSource(conf, physicalSchema, lookupConf, hbaseSchema); } private static void validatePrimaryKey(TableSchema schema) { @@ -165,9 +162,9 @@ private HBaseConf getHbaseConf(ReadableConfig config, Map option HBaseConf conf = new HBaseConf(); conf.setHbaseConfig(getHBaseClientProperties(options)); String hTableName = config.get(TABLE_NAME); - conf.setTableName(hTableName); + conf.setTable(hTableName); String nullStringLiteral = config.get(NULL_STRING_LITERAL); - conf.setNullMode(nullStringLiteral); + conf.setNullStringLiteral(nullStringLiteral); return conf; } @@ -216,19 +213,18 @@ public DynamicTableSink createDynamicTableSink(Context context) { long bufferFlushMaxSizeInBytes = config.get(SINK_BUFFER_FLUSH_MAX_SIZE).getBytes(); conf.setWriteBufferSize(bufferFlushMaxSizeInBytes); - conf.setRowkeyExpress(generateRowKey(hbaseSchema)); return new HBaseDynamicTableSink(conf, physicalSchema, hbaseSchema); } - private String generateRowKey(HBaseTableSchema hbaseSchema) { - int rowIndex = 1; - if (hbaseSchema.getRowKeyIndex() > 1) { - rowIndex = hbaseSchema.getRowKeyIndex(); - } - String familyName = hbaseSchema.getFamilyNames()[rowIndex - 1]; - String[] qualifierNames = hbaseSchema.getQualifierNames(familyName); - return Arrays.stream(qualifierNames) - .map(key -> "${" + key + "}") - .collect(Collectors.joining("_")); - } + // private String generateRowKey(HBaseTableSchema hbaseSchema) { + // int rowIndex = 1; + // if (hbaseSchema.getRowKeyIndex() > 1) { + // rowIndex = hbaseSchema.getRowKeyIndex(); + // } + // String familyName = hbaseSchema.getFamilyNames()[rowIndex - 1]; + // String[] qualifierNames = hbaseSchema.getQualifierNames(familyName); + // return Arrays.stream(qualifierNames) + // .map(key -> "${" + key + "}") + // .collect(Collectors.joining("_")); + // } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/lookup/HBaseAllTableFunction.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/lookup/HBaseAllTableFunction.java index 4e0003bf3e..945271552b 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/lookup/HBaseAllTableFunction.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/lookup/HBaseAllTableFunction.java @@ -18,11 +18,13 @@ package com.dtstack.chunjun.connector.hbase14.table.lookup; -import com.dtstack.chunjun.connector.hbase.HBaseConfigurationUtil; -import com.dtstack.chunjun.connector.hbase.HBaseSerde; import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; -import com.dtstack.chunjun.connector.hbase14.util.HBaseConfigUtils; -import com.dtstack.chunjun.lookup.AbstractAllTableFunction; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.table.lookup.AbstractHBaseAllTableFunction; +import com.dtstack.chunjun.connector.hbase.util.HBaseConfigUtils; +import com.dtstack.chunjun.connector.hbase14.converter.HBaseSerde; +import com.dtstack.chunjun.connector.hbase14.converter.HbaseRowConverter; +import com.dtstack.chunjun.connector.hbase14.util.HBaseHelper; import com.dtstack.chunjun.lookup.conf.LookupConf; import com.dtstack.chunjun.security.KerberosUtil; @@ -31,9 +33,7 @@ import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.AuthUtil; -import org.apache.hadoop.hbase.ChoreService; -import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -50,31 +50,28 @@ import java.time.LocalDateTime; import java.util.Map; -public class HBaseAllTableFunction extends AbstractAllTableFunction { +public class HBaseAllTableFunction extends AbstractHBaseAllTableFunction { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(HBaseAllTableFunction.class); - private Configuration conf; - private final byte[] serializedConfig; + private Connection conn; private String tableName; private Table table; private ResultScanner resultScanner; - private final HBaseTableSchema hbaseTableSchema; private transient HBaseSerde serde; - private final String nullStringLiteral; public HBaseAllTableFunction( - Configuration conf, - LookupConf lookupConf, - HBaseTableSchema hbaseTableSchema, - String nullStringLiteral) { - super(null, null, lookupConf, null); - this.serializedConfig = HBaseConfigurationUtil.serializeConfiguration(conf); + LookupConf lookupConf, HBaseTableSchema hbaseTableSchema, HBaseConf hBaseConf) { + super( + null, + null, + lookupConf, + new HbaseRowConverter(hbaseTableSchema, hBaseConf.getNullStringLiteral()), + hbaseTableSchema, + hBaseConf); this.tableName = hbaseTableSchema.getTableName(); - this.hbaseTableSchema = hbaseTableSchema; - this.nullStringLiteral = nullStringLiteral; } @Override @@ -85,34 +82,30 @@ public void open(FunctionContext context) throws Exception { @Override protected void loadData(Object cacheRef) { - conf = HBaseConfigurationUtil.prepareRuntimeConfiguration(serializedConfig); + Configuration hbaseDomainConf = HBaseConfiguration.create(); + for (Map.Entry entry : hBaseConf.getHbaseConfig().entrySet()) { + hbaseDomainConf.set(entry.getKey(), entry.getValue().toString()); + } + int loadDataCount = 0; try { - if (HBaseConfigUtils.isEnableKerberos(conf)) { - String principal = conf.get(HBaseConfigUtils.KEY_HBASE_CLIENT_KERBEROS_PRINCIPAL); - String keytab = conf.get(HBaseConfigUtils.KEY_HBASE_CLIENT_KEYTAB_FILE); - String krb5Conf = conf.get(HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF); + if (HBaseConfigUtils.isEnableKerberos(hbaseDomainConf)) { + String principal = + hbaseDomainConf.get(HBaseConfigUtils.KEY_HBASE_CLIENT_KERBEROS_PRINCIPAL); + String keytab = hbaseDomainConf.get(HBaseConfigUtils.KEY_HBASE_CLIENT_KEYTAB_FILE); + String krb5Conf = hbaseDomainConf.get(HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF); LOG.info("kerberos principal:{},keytab:{}", principal, keytab); System.setProperty(HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF, krb5Conf); UserGroupInformation userGroupInformation = KerberosUtil.loginAndReturnUgi(principal, keytab, krb5Conf); - Configuration finalConf = conf; + Configuration finalConf = hbaseDomainConf; conn = userGroupInformation.doAs( (PrivilegedAction) () -> { try { - ScheduledChore authChore = - AuthUtil.getAuthChore(finalConf); - if (authChore != null) { - ChoreService choreService = - new ChoreService("hbaseKerberosSink"); - choreService.scheduleChore(authChore); - } - return ConnectionFactory.createConnection( finalConf); - } catch (IOException e) { LOG.error( "Get connection fail with config:{}", @@ -120,16 +113,16 @@ protected void loadData(Object cacheRef) { throw new RuntimeException(e); } }); - + HBaseHelper.scheduleRefreshTGT(userGroupInformation); } else { - conn = ConnectionFactory.createConnection(conf); + conn = ConnectionFactory.createConnection(hbaseDomainConf); } table = conn.getTable(TableName.valueOf(tableName)); resultScanner = table.getScanner(new Scan()); Map tmpCache = (Map) cacheRef; for (Result r : resultScanner) { - tmpCache.put(serde.getRowKey(r.getRow()), serde.convertToReusedRow(r)); + tmpCache.put(serde.getRowKey(r.getRow()), serde.convertToNewRow(r)); loadDataCount++; } } catch (IOException e) { diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/lookup/HBaseLruTableFunction.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/lookup/HBaseLruTableFunction.java index f878d864a1..b2049496cd 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/lookup/HBaseLruTableFunction.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/table/lookup/HBaseLruTableFunction.java @@ -18,14 +18,15 @@ package com.dtstack.chunjun.connector.hbase14.table.lookup; -import com.dtstack.chunjun.connector.hbase.HBaseConfigurationUtil; import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.table.lookup.AbstractHBaseLruTableFunction; +import com.dtstack.chunjun.connector.hbase.util.HBaseConfigUtils; import com.dtstack.chunjun.connector.hbase14.converter.AsyncHBaseSerde; +import com.dtstack.chunjun.connector.hbase14.converter.HbaseRowConverter; import com.dtstack.chunjun.connector.hbase14.util.DtFileUtils; -import com.dtstack.chunjun.connector.hbase14.util.HBaseConfigUtils; import com.dtstack.chunjun.enums.ECacheContentType; import com.dtstack.chunjun.factory.ChunJunThreadFactory; -import com.dtstack.chunjun.lookup.AbstractLruTableFunction; import com.dtstack.chunjun.lookup.cache.CacheMissVal; import com.dtstack.chunjun.lookup.cache.CacheObj; import com.dtstack.chunjun.lookup.conf.LookupConf; @@ -51,7 +52,6 @@ import java.util.Collection; import java.util.Collections; -import java.util.Iterator; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -59,44 +59,36 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -public class HBaseLruTableFunction extends AbstractLruTableFunction { +public class HBaseLruTableFunction extends AbstractHBaseLruTableFunction { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(HBaseLruTableFunction.class); - private Config asyncClientConfig; - private Configuration conf; - private final byte[] serializedConfig; - private final String nullStringLiteral; + private static final int DEFAULT_BOSS_THREADS = 1; private static final int DEFAULT_IO_THREADS = Runtime.getRuntime().availableProcessors() * 2; private static final int DEFAULT_POOL_SIZE = DEFAULT_IO_THREADS + DEFAULT_BOSS_THREADS; + + private final String nullStringLiteral; + private transient HBaseClient hBaseClient; private String tableName; - private final HBaseTableSchema hbaseTableSchema; private transient AsyncHBaseSerde serde; public HBaseLruTableFunction( - Configuration conf, - LookupConf lookupConf, - HBaseTableSchema hbaseTableSchema, - String nullStringLiteral) { - super(lookupConf, null); - this.serializedConfig = HBaseConfigurationUtil.serializeConfiguration(conf); - this.lookupConf = lookupConf; - this.hbaseTableSchema = hbaseTableSchema; - this.nullStringLiteral = nullStringLiteral; + LookupConf lookupConf, HBaseTableSchema hbaseTableSchema, HBaseConf hBaseConf) { + super( + lookupConf, + new HbaseRowConverter(hbaseTableSchema, hBaseConf.getNullStringLiteral()), + hbaseTableSchema, + hBaseConf); + this.nullStringLiteral = hBaseConf.getNullStringLiteral(); } @Override public void open(FunctionContext context) throws Exception { super.open(context); - conf = HBaseConfigurationUtil.prepareRuntimeConfiguration(serializedConfig); - asyncClientConfig = new Config(); - Iterator> iterator = conf.iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - asyncClientConfig.overrideConfig(entry.getKey(), entry.getValue()); - } + Configuration conf = HBaseConfigUtils.getConfig(hBaseConf.getHbaseConfig()); + this.serde = new AsyncHBaseSerde(hbaseTableSchema, nullStringLiteral); tableName = hbaseTableSchema.getTableName(); ExecutorService executorService = @@ -107,6 +99,12 @@ public void open(FunctionContext context) throws Exception { TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(), new ChunJunThreadFactory("hbase-async")); + + Config asyncClientConfig = new Config(); + for (Map.Entry entry : hBaseConf.getHbaseConfig().entrySet()) { + asyncClientConfig.overrideConfig(entry.getKey(), entry.getValue().toString()); + } + if (HBaseConfigUtils.isEnableKerberos(conf)) { System.setProperty( HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF, diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/util/HBaseHelper.java b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/util/HBaseHelper.java index a944d11cd0..eb099c6dbc 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/util/HBaseHelper.java +++ b/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/util/HBaseHelper.java @@ -18,7 +18,11 @@ package com.dtstack.chunjun.connector.hbase14.util; +import com.dtstack.chunjun.connector.hbase.util.HBaseConfigUtils; import com.dtstack.chunjun.security.KerberosUtil; +import com.dtstack.chunjun.util.FileSystemUtil; + +import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -40,8 +44,11 @@ import java.io.IOException; import java.security.PrivilegedAction; import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; -import static com.dtstack.chunjun.connector.hbase14.util.HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF; +import static com.dtstack.chunjun.connector.hbase.util.HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF; import static com.dtstack.chunjun.security.KerberosUtil.KRB_STR; /** @@ -103,6 +110,7 @@ public static UserGroupInformation getUgi(Map hbaseConfigMap) KerberosUtil.loadKrb5Conf(hbaseConfigMap); KerberosUtil.refreshConfig(); + Configuration conf = FileSystemUtil.getConfiguration(hbaseConfigMap, null); return KerberosUtil.loginAndReturnUgi( principal, keytabFileName, System.getProperty(KEY_JAVA_SECURITY_KRB5_CONF)); } @@ -213,4 +221,26 @@ public static void closeBufferedMutator(BufferedMutator bufferedMutator) { throw new RuntimeException(e); } } + + public static void scheduleRefreshTGT(UserGroupInformation ugi) { + final ScheduledExecutorService executor = + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("UserGroupInformation-Relogin")); + + executor.scheduleWithFixedDelay( + new Runnable() { + + @Override + public void run() { + try { + ugi.checkTGTAndReloginFromKeytab(); + } catch (Exception e) { + LOG.error("Refresh TGT failed", e); + } + } + }, + 0, + 1, + TimeUnit.HOURS); + } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/ConstantFunction.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/ConstantFunction.java similarity index 95% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/ConstantFunction.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/ConstantFunction.java index 7191eced02..f9b5b49cfa 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/ConstantFunction.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/ConstantFunction.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.sink; +package com.dtstack.chunjun.connector.hbase; /** * @author jiangbo diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionFactory.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionFactory.java similarity index 97% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionFactory.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionFactory.java index e9528c9441..c3fb7a184e 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionFactory.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.sink; +package com.dtstack.chunjun.connector.hbase; import org.apache.commons.lang.StringUtils; diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionParser.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionParser.java similarity index 99% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionParser.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionParser.java index ce5021abdd..d920f46098 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionParser.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionParser.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.sink; +package com.dtstack.chunjun.connector.hbase; import org.apache.commons.lang.StringUtils; diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionTree.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionTree.java similarity index 94% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionTree.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionTree.java index 1db7ff4385..66c0996fb7 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/FunctionTree.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/FunctionTree.java @@ -16,13 +16,14 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.sink; +package com.dtstack.chunjun.connector.hbase; import com.google.common.collect.Lists; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang.StringUtils; +import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -31,7 +32,7 @@ * @author jiangbo * @date 2019/7/24 */ -public class FunctionTree { +public class FunctionTree implements Serializable { private String columnName; diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseColumnConverter.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseColumnConverter.java deleted file mode 100644 index 646bc835a5..0000000000 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseColumnConverter.java +++ /dev/null @@ -1,307 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.chunjun.connector.hbase; - -import com.dtstack.chunjun.conf.FieldConf; -import com.dtstack.chunjun.constants.ConstantValue; -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.column.BigDecimalColumn; -import com.dtstack.chunjun.element.column.BooleanColumn; -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.util.ColumnTypeUtil; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; - -import org.apache.hadoop.io.BytesWritable; - -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Timestamp; -import java.util.List; -import java.util.Locale; -import java.util.Map; - -/** - * @author wuren - * @program chunjun - * @create 2021/04/30 - */ -public class HBaseColumnConverter extends AbstractRowConverter { - - private List ColumnNameList; - private transient Map decimalColInfo; - - public HBaseColumnConverter(List fieldConfList) { - super(fieldConfList.size()); - for (int i = 0; i < fieldConfList.size(); i++) { - String type = fieldConfList.get(i).getType(); - int left = type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL); - int right = type.indexOf(ConstantValue.RIGHT_PARENTHESIS_SYMBOL); - if (left > 0 && right > 0) { - type = type.substring(0, left); - } - toInternalConverters.add( - wrapIntoNullableInternalConverter(createInternalConverter(type))); - toExternalConverters.add( - wrapIntoNullableExternalConverter(createExternalConverter(type), type)); - } - } - - @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; - } - - @SuppressWarnings("unchecked") - public Object[] toExternal(RowData rowData, Object[] data) throws Exception { - for (int index = 0; index < rowData.getArity(); index++) { - toExternalConverters.get(index).serialize(rowData, index, data); - } - return data; - } - - @Override - public RowData toInternalLookup(RowData input) { - throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); - } - - @Override - public Object toExternal(RowData rowData, Object output) throws Exception { - return null; - } - - @Override - @SuppressWarnings("unchecked") - protected ISerializationConverter wrapIntoNullableExternalConverter( - ISerializationConverter serializationConverter, String type) { - return (rowData, index, data) -> { - if (rowData == null || rowData.isNullAt(index)) { - data[index] = null; - } else { - serializationConverter.serialize(rowData, index, data); - } - }; - } - - @Override - @SuppressWarnings("all") - protected IDeserializationConverter createInternalConverter(String type) { - switch (type.toUpperCase(Locale.ENGLISH)) { - case "BOOLEAN": - return (IDeserializationConverter) BooleanColumn::new; - case "TINYINT": - return (IDeserializationConverter) - val -> new BigDecimalColumn(val.toString()); - case "SMALLINT": - return (IDeserializationConverter) - val -> new BigDecimalColumn(val.toString()); - case "INT": - 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 "STRING": - case "VARCHAR": - case "CHAR": - return (IDeserializationConverter) StringColumn::new; - case "TIMESTAMP": - return (IDeserializationConverter) - TimestampColumn::new; - case "DATE": - return (IDeserializationConverter) - val -> new TimestampColumn(val.getTime()); - case "BINARY": - case "VARBINARY": - return (IDeserializationConverter) BytesColumn::new; - case "TIME_WITHOUT_TIME_ZONE": - // final int timePrecision = getPrecision(type); - // if (timePrecision < MIN_TIME_PRECISION || timePrecision > - // MAX_TIME_PRECISION) { - // throw new UnsupportedOperationException( - // String.format( - // "The precision %s of TIME type is out of the - // range [%s, %s] supported by " - // + "HBase connector", - // timePrecision, MIN_TIME_PRECISION, - // MAX_TIME_PRECISION)); - // } - case "TIMESTAMP_WITHOUT_TIME_ZONE": - case "TIMESTAMP_WITH_LOCAL_TIME_ZONE": - // final int timestampPrecision = getPrecision(type); - // if (timestampPrecision < MIN_TIMESTAMP_PRECISION - // || timestampPrecision > MAX_TIMESTAMP_PRECISION) { - // throw new UnsupportedOperationException( - // String.format( - // "The precision %s of TIMESTAMP type is out of - // the range [%s, %s] supported by " - // + "HBase connector", - // timestampPrecision, - // MIN_TIMESTAMP_PRECISION, - // MAX_TIMESTAMP_PRECISION)); - // } - case "TIMESTAMP_WITH_TIME_ZONE": - case "INTERVAL_YEAR_MONTH": - case "INTERVAL_DAY_TIME": - case "ARRAY": - case "MULTISET": - case "MAP": - case "ROW": - case "STRUCTURED_TYPE": - case "DISTINCT_TYPE": - case "RAW": - case "NULL": - case "SYMBOL": - case "UNRESOLVED": - default: - throw new UnsupportedTypeException(type); - } - } - - @Override - protected ISerializationConverter createExternalConverter(String type) { - switch (type.toUpperCase(Locale.ENGLISH)) { - case "BOOLEAN": - return (rowData, index, data) -> data[index] = rowData.getBoolean(index); - case "TINYINT": - return (rowData, index, data) -> data[index] = rowData.getByte(index); - case "SMALLINT": - return (rowData, index, data) -> data[index] = rowData.getShort(index); - case "INT": - case "INTEGER": - return (rowData, index, data) -> data[index] = rowData.getInt(index); - case "BIGINT": - return (rowData, index, data) -> data[index] = rowData.getLong(index); - case "FLOAT": - return (rowData, index, data) -> data[index] = rowData.getFloat(index); - case "DOUBLE": - return (rowData, index, data) -> data[index] = rowData.getDouble(index); - case "DECIMAL": - // return (rowData, index, data) -> { - // ColumnTypeUtil.DecimalInfo decimalInfo = - // decimalColInfo.get(ColumnNameList.get(index)); - // HiveDecimal hiveDecimal = HiveDecimal.create(new - // BigDecimal(rowData.getString(index).toString())); - // hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, - // decimalInfo.getPrecision(), decimalInfo.getScale()); - // if(hiveDecimal == null){ - // String msg = String.format("The [%s] data data [%s] - // precision and scale do not match the metadata:decimal(%s, %s)", index, - // decimalInfo.getPrecision(), decimalInfo.getScale(), rowData); - // throw new WriteRecordException(msg, new - // IllegalArgumentException()); - // } - // data[index] = new HiveDecimalWritable(hiveDecimal); - // }; - case "STRING": - case "VARCHAR": - case "CHAR": - return (rowData, index, data) -> data[index] = rowData.getString(index).toString(); - case "TIMESTAMP": - return (rowData, index, data) -> - data[index] = rowData.getTimestamp(index, 6).toTimestamp(); - case "DATE": - return (rowData, index, data) -> - data[index] = new Date(rowData.getTimestamp(index, 6).getMillisecond()); - case "BINARY": - case "VARBINARY": - return (rowData, index, data) -> - data[index] = new BytesWritable(rowData.getBinary(index)); - case "TIME_WITHOUT_TIME_ZONE": - // final int timePrecision = getPrecision(type); - // if (timePrecision < MIN_TIME_PRECISION || timePrecision > - // MAX_TIME_PRECISION) { - // throw new UnsupportedOperationException( - // String.format( - // "The precision %s of TIME type is out of the - // range [%s, %s] supported by " - // + "HBase connector", - // timePrecision, MIN_TIME_PRECISION, - // MAX_TIME_PRECISION)); - // } - case "TIMESTAMP_WITHOUT_TIME_ZONE": - case "TIMESTAMP_WITH_LOCAL_TIME_ZONE": - // final int timestampPrecision = getPrecision(type); - // if (timestampPrecision < MIN_TIMESTAMP_PRECISION - // || timestampPrecision > MAX_TIMESTAMP_PRECISION) { - // throw new UnsupportedOperationException( - // String.format( - // "The precision %s of TIMESTAMP type is out of - // the range [%s, %s] supported by " - // + "HBase connector", - // timestampPrecision, - // MIN_TIMESTAMP_PRECISION, - // MAX_TIMESTAMP_PRECISION)); - // } - case "TIMESTAMP_WITH_TIME_ZONE": - case "INTERVAL_YEAR_MONTH": - case "INTERVAL_DAY_TIME": - case "ARRAY": - case "MULTISET": - case "MAP": - case "ROW": - case "STRUCTURED_TYPE": - case "DISTINCT_TYPE": - case "RAW": - case "NULL": - case "SYMBOL": - case "UNRESOLVED": - default: - throw new UnsupportedTypeException(type); - } - } - - public void setColumnNameList(List columnNameList) { - this.ColumnNameList = columnNameList; - } - - public void setDecimalColInfo(Map decimalColInfo) { - this.decimalColInfo = decimalColInfo; - } -} diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseConfigurationUtil.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseConfigurationUtil.java deleted file mode 100644 index bb7cf8af20..0000000000 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseConfigurationUtil.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.chunjun.connector.hbase; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.StringUtils; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.io.Writable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.File; -import java.io.IOException; - -/** This class helps to do serialization for hadoop Configuration and HBase-related classes. */ -@Internal -public class HBaseConfigurationUtil { - - private static final Logger LOG = LoggerFactory.getLogger(HBaseConfigurationUtil.class); - - public static final String ENV_HBASE_CONF_DIR = "HBASE_CONF_DIR"; - - public static Configuration getHBaseConfiguration() { - - // Instantiate an HBaseConfiguration to load the hbase-default.xml and hbase-site.xml from - // the classpath. - Configuration result = HBaseConfiguration.create(); - boolean foundHBaseConfiguration = false; - - // We need to load both hbase-default.xml and hbase-site.xml to the hbase configuration - // The properties of a newly added resource will override the ones in previous resources, so - // a configuration - // file with higher priority should be added later. - - // Approach 1: HBASE_HOME environment variables - String possibleHBaseConfPath = null; - - final String hbaseHome = System.getenv("HBASE_HOME"); - if (hbaseHome != null) { - LOG.debug("Searching HBase configuration files in HBASE_HOME: {}", hbaseHome); - possibleHBaseConfPath = hbaseHome + "/conf"; - } - - if (possibleHBaseConfPath != null) { - foundHBaseConfiguration = addHBaseConfIfFound(result, possibleHBaseConfPath); - } - - // Approach 2: HBASE_CONF_DIR environment variable - String hbaseConfDir = System.getenv("HBASE_CONF_DIR"); - if (hbaseConfDir != null) { - LOG.debug("Searching HBase configuration files in HBASE_CONF_DIR: {}", hbaseConfDir); - foundHBaseConfiguration = - addHBaseConfIfFound(result, hbaseConfDir) || foundHBaseConfiguration; - } - - if (!foundHBaseConfiguration) { - LOG.warn( - "Could not find HBase configuration via any of the supported methods " - + "(Flink configuration, environment variables)."); - } - - return result; - } - - /** - * Search HBase configuration files in the given path, and add them to the configuration if - * found. - */ - private static boolean addHBaseConfIfFound( - Configuration configuration, String possibleHBaseConfPath) { - boolean foundHBaseConfiguration = false; - if (new File(possibleHBaseConfPath).exists()) { - if (new File(possibleHBaseConfPath + "/hbase-default.xml").exists()) { - configuration.addResource( - new org.apache.hadoop.fs.Path( - possibleHBaseConfPath + "/hbase-default.xml")); - LOG.debug( - "Adding " - + possibleHBaseConfPath - + "/hbase-default.xml to hbase configuration"); - foundHBaseConfiguration = true; - } - if (new File(possibleHBaseConfPath + "/hbase-site.xml").exists()) { - configuration.addResource( - new org.apache.hadoop.fs.Path(possibleHBaseConfPath + "/hbase-site.xml")); - LOG.debug( - "Adding " - + possibleHBaseConfPath - + "/hbase-site.xml to hbase configuration"); - foundHBaseConfiguration = true; - } - } - return foundHBaseConfiguration; - } - - /** Serialize a Hadoop {@link Configuration} into byte[]. */ - public static byte[] serializeConfiguration(Configuration conf) { - try { - return serializeWritable(conf); - } catch (IOException e) { - throw new RuntimeException( - "Encounter an IOException when serialize the Configuration.", e); - } - } - - /** - * Deserialize a Hadoop {@link Configuration} from byte[]. Deserialize configs to {@code - * targetConfig} if it is set. - */ - public static Configuration deserializeConfiguration( - byte[] serializedConfig, Configuration targetConfig) { - if (null == targetConfig) { - targetConfig = new Configuration(); - } - try { - deserializeWritable(targetConfig, serializedConfig); - } catch (IOException e) { - throw new RuntimeException( - "Encounter an IOException when deserialize the Configuration.", e); - } - return targetConfig; - } - - /** - * Serialize writable byte[]. - * - * @param the type parameter - * @param writable the writable - * @return the byte [ ] - * @throws IOException the io exception - */ - private static byte[] serializeWritable(T writable) throws IOException { - Preconditions.checkArgument(writable != null); - - ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); - writable.write(outputStream); - return byteArrayOutputStream.toByteArray(); - } - - /** - * Deserialize writable. - * - * @param the type parameter - * @param writable the writable - * @param bytes the bytes - * @throws IOException the io exception - */ - private static void deserializeWritable(T writable, byte[] bytes) - throws IOException { - Preconditions.checkArgument(writable != null); - Preconditions.checkArgument(bytes != null); - - ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(bytes); - DataInputStream dataInputStream = new DataInputStream(byteArrayInputStream); - writable.readFields(dataInputStream); - } - - public static Configuration createHBaseConf() { - Configuration hbaseClientConf = HBaseConfiguration.create(); - - String hbaseConfDir = System.getenv(ENV_HBASE_CONF_DIR); - - if (hbaseConfDir != null) { - if (new File(hbaseConfDir).exists()) { - String coreSite = hbaseConfDir + "/core-site.xml"; - String hdfsSite = hbaseConfDir + "/hdfs-site.xml"; - String hbaseSite = hbaseConfDir + "/hbase-site.xml"; - if (new File(coreSite).exists()) { - hbaseClientConf.addResource(new org.apache.hadoop.fs.Path(coreSite)); - LOG.info("Adding " + coreSite + " to hbase configuration"); - } - if (new File(hdfsSite).exists()) { - hbaseClientConf.addResource(new org.apache.hadoop.fs.Path(hdfsSite)); - LOG.info("Adding " + hdfsSite + " to hbase configuration"); - } - if (new File(hbaseSite).exists()) { - hbaseClientConf.addResource(new org.apache.hadoop.fs.Path(hbaseSite)); - LOG.info("Adding " + hbaseSite + " to hbase configuration"); - } - } else { - LOG.warn( - "HBase config directory '{}' not found, cannot load HBase configuration.", - hbaseConfDir); - } - } else { - LOG.warn( - "{} env variable not found, cannot load HBase configuration.", - ENV_HBASE_CONF_DIR); - } - return hbaseClientConf; - } - - public static Configuration prepareRuntimeConfiguration(byte[] serializedConfig) { - // create default configuration from current runtime env (`hbase-site.xml` in classpath) - // first, - // and overwrite configuration using serialized configuration from client-side env - // (`hbase-site.xml` in classpath). - // user params from client-side have the highest priority - Configuration runtimeConfig = - HBaseConfigurationUtil.deserializeConfiguration( - serializedConfig, HBaseConfigurationUtil.getHBaseConfiguration()); - - // do validation: check key option(s) in final runtime configuration - if (StringUtils.isNullOrWhitespaceOnly(runtimeConfig.get(HConstants.ZOOKEEPER_QUORUM))) { - LOG.error( - "can not connect to HBase without {} configuration", - HConstants.ZOOKEEPER_QUORUM); - throw new IllegalArgumentException( - "check HBase configuration failed, lost: '" - + HConstants.ZOOKEEPER_QUORUM - + "'!"); - } - - return runtimeConfig; - } -} diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseConverter.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseConverter.java deleted file mode 100644 index eec88c5dae..0000000000 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseConverter.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.chunjun.connector.hbase; - -import com.dtstack.chunjun.converter.AbstractRowConverter; -import com.dtstack.chunjun.throwable.ChunJunRuntimeException; - -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; - -/** - * @author wuren - * @program chunjun - * @create 2021/04/30 - */ -public class HBaseConverter extends AbstractRowConverter { - - public HBaseConverter(RowType rowType) { - super(rowType); - 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 Object toExternal(RowData rowData, Object data) throws Exception { - for (int index = 0; index < rowData.getArity(); index++) { - toExternalConverters.get(index).serialize(rowData, index, data); - } - return data; - } -} diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseMutationConverter.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseMutationConverter.java index 7861ef3dc8..cd8697f91c 100644 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseMutationConverter.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseMutationConverter.java @@ -19,7 +19,7 @@ package com.dtstack.chunjun.connector.hbase; /** - * @program chunjun + * @program: flinkx * @author: wuren * @create: 2021/10/15 */ diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseTableSchema.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseTableSchema.java index 52c2887a49..5f1ba7cede 100644 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseTableSchema.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/HBaseTableSchema.java @@ -40,7 +40,7 @@ /** * @author wuren - * @program chunjun + * @program flinkx * @create 2021/04/30 */ diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/IFunction.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/IFunction.java similarity index 95% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/IFunction.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/IFunction.java index 9143764fbe..5553ea2459 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/IFunction.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/IFunction.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.sink; +package com.dtstack.chunjun.connector.hbase; /** * @company: www.dtstack.com diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/Md5Function.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/Md5Function.java similarity index 95% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/Md5Function.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/Md5Function.java index 48b10f9244..d204bb92b9 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/Md5Function.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/Md5Function.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.sink; +package com.dtstack.chunjun.connector.hbase; import com.dtstack.chunjun.util.Md5Util; diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/RowDataToMutationConverter.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/RowDataToMutationConverter.java deleted file mode 100644 index 5fcdabfd7d..0000000000 --- a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/RowDataToMutationConverter.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.chunjun.connector.hbase; - -/** - * @program chunjun - * @author: wuren - * @create: 2021/10/15 - */ -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.RowKind; - -import org.apache.hadoop.hbase.client.Mutation; - -/** - * An implementation of {@link HBaseMutationConverter} which converts {@link RowData} into {@link - * Mutation}. - */ -public class RowDataToMutationConverter implements HBaseMutationConverter { - private static final long serialVersionUID = 1L; - - private final HBaseTableSchema schema; - private final String nullStringLiteral; - private transient HBaseSerde serde; - - public RowDataToMutationConverter(HBaseTableSchema schema, final String nullStringLiteral) { - this.schema = schema; - this.nullStringLiteral = nullStringLiteral; - } - - @Override - public void open() { - this.serde = new HBaseSerde(schema, nullStringLiteral); - } - - @Override - public Mutation convertToMutation(RowData record) { - RowKind kind = record.getRowKind(); - if (kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER) { - return serde.createPutMutation(record); - } else { - return serde.createDeleteMutation(record); - } - } -} diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/StringFunction.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/StringFunction.java similarity index 87% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/StringFunction.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/StringFunction.java index ade7f0bea1..b72e6543f8 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/sink/StringFunction.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/StringFunction.java @@ -16,14 +16,16 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.sink; +package com.dtstack.chunjun.connector.hbase; + +import java.io.Serializable; /** * @company: www.dtstack.com * @author: toutian * @create: 2019/7/23 */ -public class StringFunction implements IFunction { +public class StringFunction implements IFunction, Serializable { @Override public String evaluate(Object str) { diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConf.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConf.java similarity index 85% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConf.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConf.java index 41df9b6e32..5bc6ccb0e8 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConf.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConf.java @@ -13,18 +13,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.conf; +package com.dtstack.chunjun.connector.hbase.conf; import com.dtstack.chunjun.conf.ChunJunCommonConf; -import com.dtstack.chunjun.conf.FieldConf; -import java.util.List; import java.util.Map; public class HBaseConf extends ChunJunCommonConf { - - private List columnMetaInfos; - private String encoding; + private String encoding = "UTF-8"; private Map hbaseConfig; // reader @@ -36,20 +32,13 @@ public class HBaseConf extends ChunJunCommonConf { // writer private String nullMode; - private Boolean walFlag; + private String nullStringLiteral; + private Boolean walFlag = false; private long writeBufferSize; private String rowkeyExpress; private Integer versionColumnIndex; private String versionColumnValue; - public List getColumnMetaInfos() { - return columnMetaInfos; - } - - public void setColumnMetaInfos(List columnMetaInfos) { - this.columnMetaInfos = columnMetaInfos; - } - public String getEncoding() { return encoding; } @@ -98,14 +87,6 @@ public void setTable(String table) { this.table = table; } - public void setTableName(String tableName) { - this.table = tableName; - } - - public String getTableName() { - return table; - } - public int getScanCacheSize() { return scanCacheSize; } @@ -161,4 +142,12 @@ public String getVersionColumnValue() { public void setVersionColumnValue(String versionColumnValue) { this.versionColumnValue = versionColumnValue; } + + public String getNullStringLiteral() { + return nullStringLiteral; + } + + public void setNullStringLiteral(String nullStringLiteral) { + this.nullStringLiteral = nullStringLiteral; + } } diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConfigConstants.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConfigConstants.java similarity index 96% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConfigConstants.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConfigConstants.java index f6ccdfea37..99126069db 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConfigConstants.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConfigConstants.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.conf; +package com.dtstack.chunjun.connector.hbase.conf; /** * The class containing Hbase configuration constants diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConfigKeys.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConfigKeys.java similarity index 97% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConfigKeys.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConfigKeys.java index abe2cf1674..9e65c1edb8 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/conf/HBaseConfigKeys.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/conf/HBaseConfigKeys.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.conf; +package com.dtstack.chunjun.connector.hbase.conf; /** * This class defines configuration keys for HbaseReader and HbaseWriter diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseRawTypeConverter.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/converter/HBaseRawTypeConverter.java similarity index 66% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseRawTypeConverter.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/converter/HBaseRawTypeConverter.java index b73a9b594a..2881588e8f 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/converter/HBaseRawTypeConverter.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/converter/HBaseRawTypeConverter.java @@ -1,31 +1,35 @@ /* - * Copyright 2021 the original author or authors. + * 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 * - * Licensed 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 * - * 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. + * 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.hbase14.converter; -import com.dtstack.chunjun.converter.RawTypeConverter; +package com.dtstack.chunjun.connector.hbase.converter; + +import com.dtstack.chunjun.connector.hbase.converter.type.BINARYSTRING; import com.dtstack.chunjun.throwable.UnsupportedTypeException; import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.AtomicDataType; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; import java.util.Locale; -public class HBaseRawTypeConverter implements RawTypeConverter { - - public DataType apply(String type) { +public class HBaseRawTypeConverter { + public static DataType apply(String type) { switch (type.toUpperCase(Locale.ENGLISH)) { case "BOOLEAN": return DataTypes.BOOLEAN(); @@ -33,7 +37,10 @@ public DataType apply(String type) { case "INT8": case "UINT8": return DataTypes.TINYINT(); + case "BINARY_STRING": + return new AtomicDataType(new BINARYSTRING(true, LogicalTypeRoot.VARCHAR)); case "SMALLINT": + case "SHORT": case "UINT16": case "INT16": return DataTypes.SMALLINT(); @@ -53,6 +60,7 @@ public DataType apply(String type) { case "UINT64": case "INT64": case "BIGINT": + case "LONG": return DataTypes.BIGINT(); case "FLOAT": case "FLOAT32": diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/converter/type/BINARYSTRING.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/converter/type/BINARYSTRING.java new file mode 100644 index 0000000000..296bed7455 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/converter/type/BINARYSTRING.java @@ -0,0 +1,78 @@ +/* + * 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.hbase.converter.type; + +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.LogicalTypeVisitor; + +import java.io.Reader; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +public class BINARYSTRING extends LogicalType { + + private static final Set INPUT_CONVERSION = + conversionSet(String.class.getName(), StringData.class.getName()); + + private static final Class DEFAULT_CONVERSION = String.class; + + private static final Set OUTPUT_CONVERSION = conversionSet(Reader.class.getName()); + + public BINARYSTRING(boolean isNullable, LogicalTypeRoot typeRoot) { + super(isNullable, typeRoot); + } + + @Override + public String asSerializableString() { + return "HBASE-Biary-String"; + } + + @Override + public boolean supportsInputConversion(Class clazz) { + return INPUT_CONVERSION.contains(clazz.getName()); + } + + @Override + public boolean supportsOutputConversion(Class clazz) { + return OUTPUT_CONVERSION.contains(clazz.getName()); + } + + @Override + public Class getDefaultConversion() { + return DEFAULT_CONVERSION; + } + + @Override + public List getChildren() { + return Collections.emptyList(); + } + + @Override + public R accept(LogicalTypeVisitor visitor) { + return visitor.visit(this); + } + + @Override + public LogicalType copy(boolean isNullable) { + return new BINARYSTRING(isNullable, getTypeRoot()); + } +} diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/BaseHBaseDynamicTableSource.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/BaseHBaseDynamicTableSource.java new file mode 100644 index 0000000000..ed4eba350b --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/BaseHBaseDynamicTableSource.java @@ -0,0 +1,142 @@ +/* + * 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.hbase.table; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.connector.hbase.table.lookup.AbstractHBaseAllTableFunction; +import com.dtstack.chunjun.connector.hbase.util.HBaseConfigUtils; +import com.dtstack.chunjun.enums.CacheType; +import com.dtstack.chunjun.lookup.AbstractLruTableFunction; +import com.dtstack.chunjun.lookup.conf.LookupConf; +import com.dtstack.chunjun.source.DtInputFormatSourceFunction; +import com.dtstack.chunjun.source.format.BaseRichInputFormatBuilder; +import com.dtstack.chunjun.table.connector.source.ParallelAsyncTableFunctionProvider; +import com.dtstack.chunjun.table.connector.source.ParallelSourceFunctionProvider; +import com.dtstack.chunjun.table.connector.source.ParallelTableFunctionProvider; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.LookupTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.Preconditions; + +import java.util.ArrayList; +import java.util.List; + +public abstract class BaseHBaseDynamicTableSource + implements ScanTableSource, LookupTableSource, SupportsProjectionPushDown { + protected TableSchema tableSchema; + protected HBaseTableSchema hbaseSchema; + + protected final HBaseConf hBaseConf; + protected final LookupConf lookupConf; + + public BaseHBaseDynamicTableSource( + TableSchema tableSchema, + HBaseTableSchema hbaseSchema, + HBaseConf hBaseConf, + LookupConf lookupConf) { + this.tableSchema = tableSchema; + this.hbaseSchema = hbaseSchema; + this.hBaseConf = hBaseConf; + this.hbaseSchema.setTableName(hBaseConf.getTable()); + this.lookupConf = lookupConf; + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + final RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + TypeInformation typeInformation = InternalTypeInfo.of(rowType); + + String[] fieldNames = tableSchema.getFieldNames(); + List columnList = new ArrayList<>(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + FieldConf field = new FieldConf(); + field.setName(fieldNames[i]); + field.setType(rowType.getTypeAt(i).asSummaryString()); + field.setIndex(i); + columnList.add(field); + } + hBaseConf.setColumn(columnList); + + BaseRichInputFormatBuilder builder = getBaseRichInputFormatBuilder(); + + return ParallelSourceFunctionProvider.of( + new DtInputFormatSourceFunction<>(builder.finish(), typeInformation), + true, + hBaseConf.getParallelism()); + } + + @Override + public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { + String[] keyNames = new String[context.getKeys().length]; + for (int i = 0; i < keyNames.length; i++) { + int[] innerKeyArr = context.getKeys()[i]; + Preconditions.checkArgument( + innerKeyArr.length == 1, "redis only support non-nested look up keys"); + keyNames[i] = tableSchema.getFieldNames()[innerKeyArr[0]]; + } + fillKerberosConf(); + hbaseSchema.setTableName(hBaseConf.getTable()); + if (lookupConf.getCache().equalsIgnoreCase(CacheType.LRU.toString())) { + return ParallelAsyncTableFunctionProvider.of( + getAbstractLruTableFunction(), lookupConf.getParallelism()); + } + return ParallelTableFunctionProvider.of( + getAbstractAllTableFunction(), lookupConf.getParallelism()); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public boolean supportsNestedProjection() { + return false; + } + + @Override + public void applyProjection(int[][] projectedFields) { + TableSchema projectSchema = + TableSchemaUtils.projectSchema( + hbaseSchema.convertsToTableSchema(), projectedFields); + this.hbaseSchema = HBaseTableSchema.fromTableSchema(projectSchema); + } + + protected abstract BaseRichInputFormatBuilder getBaseRichInputFormatBuilder(); + + protected abstract AbstractLruTableFunction getAbstractLruTableFunction(); + + protected abstract AbstractHBaseAllTableFunction getAbstractAllTableFunction(); + + private void fillKerberosConf() { + if (HBaseConfigUtils.isEnableKerberos(hBaseConf.getHbaseConfig())) { + HBaseConfigUtils.fillKerberosConfig(hBaseConf.getHbaseConfig()); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/lookup/AbstractHBaseAllTableFunction.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/lookup/AbstractHBaseAllTableFunction.java new file mode 100644 index 0000000000..5400b31ef6 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/lookup/AbstractHBaseAllTableFunction.java @@ -0,0 +1,46 @@ +/* + * 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.hbase.table.lookup; + +import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.lookup.AbstractAllTableFunction; +import com.dtstack.chunjun.lookup.conf.LookupConf; + +public abstract class AbstractHBaseAllTableFunction extends AbstractAllTableFunction { + + protected final HBaseTableSchema hbaseTableSchema; + protected final HBaseConf hBaseConf; + + protected String nullStringLiteral; + + public AbstractHBaseAllTableFunction( + String[] fieldNames, + String[] keyNames, + LookupConf lookupConf, + AbstractRowConverter rowConverter, + HBaseTableSchema hbaseTableSchema, + HBaseConf hBaseConf) { + super(fieldNames, keyNames, lookupConf, rowConverter); + this.hbaseTableSchema = hbaseTableSchema; + this.hBaseConf = hBaseConf; + this.nullStringLiteral = hBaseConf.getNullStringLiteral(); + } +} diff --git a/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/lookup/AbstractHBaseLruTableFunction.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/lookup/AbstractHBaseLruTableFunction.java new file mode 100644 index 0000000000..11058cb2a7 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/table/lookup/AbstractHBaseLruTableFunction.java @@ -0,0 +1,41 @@ +/* + * 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.hbase.table.lookup; + +import com.dtstack.chunjun.connector.hbase.HBaseTableSchema; +import com.dtstack.chunjun.connector.hbase.conf.HBaseConf; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.lookup.AbstractLruTableFunction; +import com.dtstack.chunjun.lookup.conf.LookupConf; + +public abstract class AbstractHBaseLruTableFunction extends AbstractLruTableFunction { + + protected final HBaseTableSchema hbaseTableSchema; + protected final HBaseConf hBaseConf; + + public AbstractHBaseLruTableFunction( + LookupConf lookupConf, + AbstractRowConverter rowConverter, + HBaseTableSchema hbaseTableSchema, + HBaseConf hBaseConf) { + super(lookupConf, rowConverter); + this.hbaseTableSchema = hbaseTableSchema; + this.hBaseConf = hBaseConf; + } +} diff --git a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/util/HBaseConfigUtils.java b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/util/HBaseConfigUtils.java similarity index 89% rename from chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/util/HBaseConfigUtils.java rename to chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/util/HBaseConfigUtils.java index ad5a7d9d8d..438a3b2c7e 100644 --- a/chunjun-connectors/chunjun-connector-hbase-1.4/src/main/java/com/dtstack/chunjun/connector/hbase14/util/HBaseConfigUtils.java +++ b/chunjun-connectors/chunjun-connector-hbase-base/src/main/java/com/dtstack/chunjun/connector/hbase/util/HBaseConfigUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.chunjun.connector.hbase14.util; +package com.dtstack.chunjun.connector.hbase.util; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -24,7 +24,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.hbase.async.Config; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -136,8 +135,7 @@ public static String getPrincipal(Map hbaseConfigMap) { throw new IllegalArgumentException(KEY_PRINCIPAL + " is not set!"); } - public static void fillSyncKerberosConfig( - Configuration config, Map hbaseConfigMap) { + public static void fillKerberosConfig(Map hbaseConfigMap) { if (StringUtils.isEmpty( MapUtils.getString(hbaseConfigMap, KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL))) { throw new IllegalArgumentException( @@ -146,21 +144,34 @@ public static void fillSyncKerberosConfig( String regionServerPrincipal = MapUtils.getString(hbaseConfigMap, KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL); - config.set(HBaseConfigUtils.KEY_HBASE_MASTER_KERBEROS_PRINCIPAL, regionServerPrincipal); - config.set( + + hbaseConfigMap.put( HBaseConfigUtils.KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL, regionServerPrincipal); - config.set(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, "true"); - config.set(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHENTICATION, KRB_STR); - config.set(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, KRB_STR); + + hbaseConfigMap.put(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, "true"); + hbaseConfigMap.put(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHENTICATION, KRB_STR); + hbaseConfigMap.put(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, KRB_STR); if (!StringUtils.isEmpty(MapUtils.getString(hbaseConfigMap, KEY_ZOOKEEPER_SASL_CLIENT))) { System.setProperty( HBaseConfigUtils.KEY_ZOOKEEPER_SASL_CLIENT, MapUtils.getString(hbaseConfigMap, KEY_ZOOKEEPER_SASL_CLIENT)); } + + String principal = HBaseConfigUtils.getPrincipal(hbaseConfigMap); + + String keytab = + HBaseConfigUtils.loadKeyFromConf(hbaseConfigMap, HBaseConfigUtils.KEY_KEY_TAB); + String krb5Conf = + HBaseConfigUtils.loadKeyFromConf( + hbaseConfigMap, HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF); + hbaseConfigMap.put(HBaseConfigUtils.KEY_HBASE_CLIENT_KEYTAB_FILE, keytab); + hbaseConfigMap.put(HBaseConfigUtils.KEY_HBASE_CLIENT_KERBEROS_PRINCIPAL, principal); + hbaseConfigMap.put(HBaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF, krb5Conf); } - public static void fillSyncKerberosConfig(Config config, Map hbaseConfigMap) { + public static void fillSyncKerberosConfig( + Configuration config, Map hbaseConfigMap) { if (StringUtils.isEmpty( MapUtils.getString(hbaseConfigMap, KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL))) { throw new IllegalArgumentException( @@ -169,19 +180,12 @@ public static void fillSyncKerberosConfig(Config config, Map hba String regionServerPrincipal = MapUtils.getString(hbaseConfigMap, KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL); - config.overrideConfig( - HBaseConfigUtils.KEY_HBASE_MASTER_KERBEROS_PRINCIPAL, regionServerPrincipal); - config.overrideConfig( + config.set(HBaseConfigUtils.KEY_HBASE_MASTER_KERBEROS_PRINCIPAL, regionServerPrincipal); + config.set( HBaseConfigUtils.KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL, regionServerPrincipal); - config.overrideConfig(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, "true"); - config.overrideConfig(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHENTICATION, KRB_STR); - config.overrideConfig(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, KRB_STR); - - if (!StringUtils.isEmpty(MapUtils.getString(hbaseConfigMap, KEY_ZOOKEEPER_SASL_CLIENT))) { - System.setProperty( - HBaseConfigUtils.KEY_ZOOKEEPER_SASL_CLIENT, - MapUtils.getString(hbaseConfigMap, KEY_ZOOKEEPER_SASL_CLIENT)); - } + config.set(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, "true"); + config.set(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHENTICATION, KRB_STR); + config.set(HBaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, KRB_STR); } public static void loadKrb5Conf(Map config) { diff --git a/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/sink/HdfsOrcOutputFormat.java b/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/sink/HdfsOrcOutputFormat.java index d365e8b8e3..e4cff5c448 100644 --- a/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/sink/HdfsOrcOutputFormat.java +++ b/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/sink/HdfsOrcOutputFormat.java @@ -75,6 +75,8 @@ public class HdfsOrcOutputFormat extends BaseHdfsOutputFormat { private FileOutputFormat outputFormat; private JobConf jobConf; + protected int[] colIndices; + /** 初始化对象大小计算器 */ protected void initRowSizeCalculator() { rowSizeCalculator = RowSizeCalculator.getRowSizeCalculator(); @@ -108,17 +110,16 @@ protected void openSource() { } FileOutputFormat.setOutputCompressorClass(jobConf, codecClass); - int size = hdfsConf.getColumn().size(); + int size = hdfsConf.getFullColumnType().size(); decimalColInfo = Maps.newHashMapWithExpectedSize(size); List structFieldObjectInspectors = new ArrayList<>(); for (int i = 0; i < size; i++) { - FieldConf fieldConf = hdfsConf.getColumn().get(i); - String columnType = fieldConf.getType(); + String columnType = hdfsConf.getFullColumnType().get(i); if (ColumnTypeUtil.isDecimalType(columnType)) { ColumnTypeUtil.DecimalInfo decimalInfo = ColumnTypeUtil.getDecimalInfo(columnType, ORC_DEFAULT_DECIMAL_INFO); - decimalColInfo.put(fieldConf.getName(), decimalInfo); + decimalColInfo.put(hdfsConf.getFullColumnName().get(i), decimalInfo); } ColumnType type = ColumnType.getType(columnType); structFieldObjectInspectors.add(HdfsUtil.columnTypeToObjectInspetor(type)); @@ -135,6 +136,22 @@ protected void openSource() { this.inspector = ObjectInspectorFactory.getStandardStructObjectInspector( fullColumnNameList, structFieldObjectInspectors); + + colIndices = new int[hdfsConf.getFullColumnName().size()]; + for (int i = 0; i < hdfsConf.getFullColumnName().size(); ++i) { + int j = 0; + for (; j < hdfsConf.getColumn().size(); ++j) { + if (hdfsConf.getFullColumnName() + .get(i) + .equalsIgnoreCase(hdfsConf.getColumn().get(j).getName())) { + colIndices[i] = j; + break; + } + } + if (j == hdfsConf.getColumn().size()) { + colIndices[i] = -1; + } + } } @Override @@ -204,8 +221,18 @@ public void writeSingleRecordToFile(RowData rowData) throws WriteRecordException } try { + List recordList = new ArrayList<>(); + for (int i = 0; i < hdfsConf.getFullColumnName().size(); ++i) { + int colIndex = colIndices[i]; + if (colIndex == -1) { + recordList.add(null); + } else { + recordList.add(data[colIndex]); + } + } + this.recordWriter.write( - NullWritable.get(), this.orcSerde.serialize(data, this.inspector)); + NullWritable.get(), this.orcSerde.serialize(recordList, this.inspector)); rowsOfCurrentBlock++; lastRow = rowData; } catch (IOException e) { diff --git a/chunjun-connectors/chunjun-connector-inceptor/pom.xml b/chunjun-connectors/chunjun-connector-inceptor/pom.xml index fd841a2549..c3ca3d1569 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/pom.xml +++ b/chunjun-connectors/chunjun-connector-inceptor/pom.xml @@ -13,13 +13,158 @@ ChunJun : Connectors : Inceptor + + io.transwarp.wrdecimal + wrdecimal-library + 0.1 + + + + inceptor-serde + org.apache.hive + 8.0.1 + + + + + org.apache.hive + inceptor-exec + 8.0.1 + + + + + org.apache.hive + inceptor-metastore + 8.0.1 + + + io.transwarp inceptor-driver 6.0.2 + + + org.apache.hadoop + hadoop-hdfs + + + commons-cli + commons-cli + + + + commons-cli + commons-cli + 1.3.1 + + + + parquet-hadoop + org.apache.parquet + 1.8.3 + + + org.xerial.snappy + snappy-java + + + commons-codec + commons-codec + + + jackson-core-asl + org.codehaus.jackson + + + jackson-mapper-asl + org.codehaus.jackson + + + slf4j-api + org.slf4j + + + + + + org.apache.thrift + libfb303 + 0.9.2 + + + httpclient + org.apache.httpcomponents + + + httpcore + org.apache.httpcomponents + + + slf4j-api + org.slf4j + + + + + + + org.apache.hive + inceptor-streaming + 3.1.3 + + + hadoop-hdfs + org.apache.hadoop + + + hadoop-common + org.apache.hadoop + + + hadoop-auth + org.apache.hadoop + + + hadoop-annotations + org.apache.hadoop + + + hadoop-yarn-api + org.apache.hadoop + + + hadoop-yarn-common + org.apache.hadoop + + + hive-serde + org.apache.hive + + + calcite-core + org.apache.calcite + + + calcite-avatica + org.apache.calcite + + + derby + org.apache.derby + + + org.xerial.snappy + snappy-java + + + + + com.dtstack.chunjun chunjun-connector-jdbc-base @@ -29,8 +174,9 @@ org.apache.hadoop hadoop-mapreduce-client-core - 2.7.5 + ${hadoop.version} + @@ -58,14 +204,82 @@ + + + com.facebook.fb303 + shade.inceptor.com.facebook.fb303 + + + + org.apache.hadoop.hive + shade.inceptor.org.apache.hadoop.hive + + org.apache.hadoop.hive.ql.io.ProxyLocalFileSystem + org.apache.hadoop.hive.ql.io.NullScanFileSystem + org.apache.hadoop.hive.llap.security.* + + + + org.apache.commons.cli shade.inceptor.org.apache.commons.cli + + org.apache.hadoop shade.inceptor.org.apache.hadoop + + org.apache.hadoop.security.GroupMappingServiceProvider + org.apache.hadoop.fs.* + + org.apache.hadoop.conf.* + + org.apache.hadoop.hdfs.* + org.apache.hadoop.hdfs.protocol.* + org.apache.hadoop.hdfs.protocol.datatransfer.* + org.apache.hadoop.hdfs.protocol.proto.* + org.apache.hadoop.hdfs.protocolPB.* + org.apache.hadoop.mapred.* + org.apache.hadoop.mapreduce.* + org.apache.hadoop.filecache.* + + org.apache.hadoop.io.retry.* + org.apache.hadoop.hdfs.server.* + org.apache.hadoop.hdfs.server.namenode.* + org.apache.hadoop.hdfs.server.namenode.ha.* + org.apache.hadoop.hdfs.server.protocol.* + org.apache.hadoop.util.NativeCrc32 + org.apache.hadoop.util.DataChecksum* + + + + + org.apache.hadoop.mapred.FileOutputFormat + shade.inceptor.org.apache.hadoop.mapred.FileOutputFormat + + + + org.apache.hadoop.mapred.TextInputFormat + shade.inceptor.org.apache.hadoop.mapred.TextInputFormat + + + + org.apache.hadoop.mapred.LineRecordReader + shade.inceptor.org.apache.hadoop.mapred.LineRecordReader + + + + org.apache.hadoop.mapred.KeyValueLineRecordReader + shade.inceptor.org.apache.hadoop.mapred.KeyValueLineRecordReader + + + + org.apache.hadoop.hdfs.HAUtil + shade.inceptor.org.apache.hadoop.hdfs.HAUtil + org.apache.hive shade.inceptor.org.apache.hive @@ -82,6 +296,10 @@ com.esotericsoftware.kryo shade.inceptor.com.esotericsoftware.kryo + + com.google.protobuf + shade.inceptor.com.google.protobuf + getHadoopConfig() { public void setHadoopConfig(Map hadoopConfig) { this.hadoopConfig = hadoopConfig; } + + public String getDriverType() { + return driverType; + } + + public void setDriverType(String driverType) { + this.driverType = driverType; + } } diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/conf/InceptorFileConf.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/conf/InceptorFileConf.java new file mode 100644 index 0000000000..a96627d5e5 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/conf/InceptorFileConf.java @@ -0,0 +1,160 @@ +/* + * 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.inceptor.conf; + +import com.dtstack.chunjun.conf.BaseFileConf; + +import org.apache.parquet.hadoop.ParquetWriter; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class InceptorFileConf extends BaseFileConf { + + protected String defaultFs; + protected String fileType; + protected String fieldDelimiter; + protected Map hadoopConfig = new HashMap<>(); + protected String filterRegex; + protected boolean transaction = false; + + // write + protected int rowGroupSize = ParquetWriter.DEFAULT_BLOCK_SIZE; + private List fullColumnName; + private List fullColumnType; + private boolean enableDictionary = true; + private String schema; + private String table; + private String partitionName; + /** 目前只支持UTF-8 */ + protected String charsetName = "UTF-8"; + + 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 String getFieldDelimiter() { + return fieldDelimiter; + } + + public void setFieldDelimiter(String fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + } + + 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 boolean isTransaction() { + return transaction; + } + + public void setTransaction(boolean transaction) { + this.transaction = transaction; + } + + public String getCharsetName() { + return charsetName; + } + + public void setCharsetName(String charsetName) { + this.charsetName = charsetName; + } + + 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 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 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 String getPartitionName() { + return partitionName; + } + + public void setPartitionName(String partitionName) { + this.partitionName = partitionName; + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorOrcColumnConvent.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorOrcColumnConvent.java new file mode 100644 index 0000000000..ed05aeeb86 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorOrcColumnConvent.java @@ -0,0 +1,306 @@ +/* + * 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.inceptor.converter; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.constants.ConstantValue; +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.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.ColumnTypeUtil; +import com.dtstack.chunjun.util.DateUtil; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; + +import org.apache.hadoop.hive.common.Dialect; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.HiveVarchar2Writable; +import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.io.BytesWritable; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +public class InceptorOrcColumnConvent + extends AbstractRowConverter { + + private List ColumnNameList; + private transient Map decimalColInfo; + + private boolean columnIsStarSymbol; + + public InceptorOrcColumnConvent(List fieldConfList) { + super(fieldConfList.size()); + columnIsStarSymbol = + fieldConfList.size() == 1 + && ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName()); + if (!columnIsStarSymbol) { + for (int i = 0; i < fieldConfList.size(); i++) { + String type = fieldConfList.get(i).getType(); + int left = type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL); + int right = type.indexOf(ConstantValue.RIGHT_PARENTHESIS_SYMBOL); + if (left > 0 && right > 0) { + type = type.substring(0, left); + } + toInternalConverters.add( + wrapIntoNullableInternalConverter(createInternalConverter(type))); + toExternalConverters.add( + wrapIntoNullableExternalConverter(createExternalConverter(type), type)); + } + } + } + + @Override + public RowData toInternal(RowData input) throws Exception { + ColumnRowData row = new ColumnRowData(input.getArity()); + if (input instanceof GenericRowData) { + GenericRowData genericRowData = (GenericRowData) input; + if (columnIsStarSymbol) { + for (int i = 0; i < input.getArity(); i++) { + if (genericRowData.getField(i) == null) { + row.addField(null); + } else { + row.addField(new StringColumn(genericRowData.getField(i).toString())); + } + } + } else { + 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 + public Object[] toExternal(RowData rowData, Object[] output) throws Exception { + for (int index = 0; index < rowData.getArity(); index++) { + toExternalConverters.get(index).serialize(rowData, index, output); + } + return output; + } + + @Override + @SuppressWarnings("unchecked") + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, String type) { + return (rowData, index, data) -> { + if (rowData == null || rowData.isNullAt(index)) { + data[index] = null; + } else { + serializationConverter.serialize(rowData, index, data); + } + }; + } + + @Override + @SuppressWarnings("all") + protected IDeserializationConverter createInternalConverter(String type) { + switch (type.toUpperCase(Locale.ENGLISH)) { + case "BOOLEAN": + return val -> new BooleanColumn(Boolean.valueOf(val.toString())); + case "TINYINT": + return val -> new BigDecimalColumn(val.toString()); + case "SMALLINT": + return val -> new BigDecimalColumn(val.toString()); + case "INT": + return val -> new BigDecimalColumn(val.toString()); + case "BIGINT": + return val -> new BigDecimalColumn(val.toString()); + case "FLOAT": + return val -> new BigDecimalColumn(val.toString()); + case "DOUBLE": + return val -> new BigDecimalColumn(val.toString()); + case "DECIMAL": + return val -> new BigDecimalColumn(val.toString()); + case "STRING": + case "VARCHAR": + case "CHAR": + return val -> new StringColumn(val.toString()); + case "TIMESTAMP": + return (IDeserializationConverter) + val -> + new TimestampColumn( + val, DateUtil.getPrecisionFromTimestampStr(val.toString())); + case "DATE": + return val -> { + if (val instanceof java.util.Date) { + return new TimestampColumn(((java.util.Date) val).getTime()); + } else { + return new TimestampColumn(((java.sql.Date) val).getTime()); + } + }; + case "BINARY": + return (IDeserializationConverter) BytesColumn::new; + case "ARRAY": + case "MAP": + case "STRUCT": + case "UNION": + default: + throw new UnsupportedTypeException(type); + } + } + + @Override + protected ISerializationConverter createExternalConverter(String type) { + String previousType = type; + if (type.contains(ConstantValue.LEFT_PARENTHESIS_SYMBOL)) { + type = type.substring(0, type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL)); + } + + switch (type.toUpperCase(Locale.ENGLISH)) { + case "BOOLEAN": + return (rowData, index, data) -> data[index] = rowData.getBoolean(index); + case "TINYINT": + return (rowData, index, data) -> + data[index] = new ByteWritable(rowData.getByte(index)); + case "SMALLINT": + return (rowData, index, data) -> + data[index] = new ShortWritable(rowData.getShort(index)); + case "INT": + return (rowData, index, data) -> data[index] = rowData.getInt(index); + case "BIGINT": + return (rowData, index, data) -> data[index] = rowData.getLong(index); + case "FLOAT": + return (rowData, index, data) -> data[index] = rowData.getFloat(index); + case "DOUBLE": + return (rowData, index, data) -> + data[index] = new DoubleWritable(rowData.getDouble(index)); + case "DECIMAL": + return (rowData, index, data) -> { + ColumnTypeUtil.DecimalInfo decimalInfo = + decimalColInfo.get(ColumnNameList.get(index)); + HiveDecimal hiveDecimal = + HiveDecimal.create(new BigDecimal(rowData.getString(index).toString())); + hiveDecimal = + HiveDecimal.enforcePrecisionScale( + hiveDecimal, + decimalInfo.getPrecision(), + decimalInfo.getScale()); + if (hiveDecimal == null) { + String msg = + String.format( + "The [%s] data data [%s] precision and scale do not match the metadata:decimal(%s, %s)", + index, + decimalInfo.getPrecision(), + decimalInfo.getScale(), + rowData); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + data[index] = new HiveDecimalWritable(hiveDecimal); + }; + case "STRING": + return (rowData, index, data) -> { + if (rowData instanceof ColumnRowData) { + Object columnData = ((ColumnRowData) rowData).getField(index).getData(); + if (columnData instanceof Timestamp) { + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); + data[index] = fm.format(columnData); + } else { + data[index] = rowData.getString(index).toString(); + } + } else { + data[index] = rowData.getString(index).toString(); + } + }; + + case "VARCHAR": + return (rowData, index, data) -> { + HiveVarchar hiveVarchar = + new HiveVarchar( + rowData.getString(index).toString(), + getVarcharLength(previousType)); + HiveVarcharWritable hiveVarcharWritable = new HiveVarcharWritable(hiveVarchar); + data[index] = hiveVarcharWritable; + }; + case "VARCHAR2": + return (rowData, index, data) -> + data[index] = + HiveVarchar2Writable.createInstance( + Dialect.UNKNOWN, + getVarcharLength(previousType), + rowData.getString(index).toString()); + case "TIMESTAMP": + return (rowData, index, data) -> + data[index] = rowData.getTimestamp(index, 6).toTimestamp(); + case "DATE": + return (rowData, index, data) -> + data[index] = + new DateWritable(rowData.getTimestamp(index, 6).getMillisecond()); + case "BINARY": + return (rowData, index, data) -> + data[index] = new BytesWritable(rowData.getBinary(index)); + case "ARRAY": + case "MAP": + case "STRUCT": + case "UNION": + default: + throw new UnsupportedTypeException(type); + } + } + + public void setColumnNameList(List columnNameList) { + this.ColumnNameList = columnNameList; + } + + public void setDecimalColInfo(Map decimalColInfo) { + this.decimalColInfo = decimalColInfo; + } + + private int getVarcharLength(String columnName) { + if (columnName.contains(ConstantValue.LEFT_PARENTHESIS_SYMBOL)) { + return Integer.parseInt( + columnName.substring( + columnName.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL) + 1, + columnName.indexOf(ConstantValue.RIGHT_PARENTHESIS_SYMBOL))); + } + return -1; + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorOrcRowConverter.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorOrcRowConverter.java new file mode 100644 index 0000000000..a603037437 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorOrcRowConverter.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.inceptor.converter; + +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.LogicalTypeRoot; +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.time.LocalDate; + +public class InceptorOrcRowConverter + extends AbstractRowConverter { + + public InceptorOrcRowConverter(RowType rowType) { + super(rowType); + 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 Object[] toExternal(RowData rowData, Object[] data) throws Exception { + for (int index = 0; index < rowData.getArity(); index++) { + toExternalConverters.get(index).serialize(rowData, index, data); + } + return data; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); + } + + @Override + @SuppressWarnings("unchecked") + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, LogicalType type) { + return (rowData, index, data) -> { + if (rowData == null + || rowData.isNullAt(index) + || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) { + data[index] = null; + } else { + serializationConverter.serialize(rowData, index, 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[index] = null; + case BOOLEAN: + return (rowData, index, data) -> data[index] = rowData.getBoolean(index); + case TINYINT: + return (rowData, index, data) -> data[index] = rowData.getByte(index); + case SMALLINT: + return (rowData, index, data) -> data[index] = rowData.getShort(index); + case INTEGER: + return (rowData, index, data) -> data[index] = rowData.getInt(index); + case BIGINT: + return (rowData, index, data) -> data[index] = rowData.getLong(index); + case DATE: + return (rowData, index, data) -> { + data[index] = Date.valueOf(LocalDate.ofEpochDay(rowData.getInt(index))); + }; + case FLOAT: + return (rowData, index, data) -> data[index] = rowData.getFloat(index); + case DOUBLE: + return (rowData, index, data) -> data[index] = rowData.getDouble(index); + case CHAR: + case VARCHAR: + return (rowData, index, data) -> data[index] = 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[index] = new HiveDecimalWritable(hiveDecimal); + }; + case BINARY: + case VARBINARY: + return (rowData, index, data) -> + data[index] = new BytesWritable(rowData.getBinary(index)); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, data) -> + data[index] = + 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-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorParquetColumnConverter.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorParquetColumnConverter.java new file mode 100644 index 0000000000..d242d1bc69 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorParquetColumnConverter.java @@ -0,0 +1,259 @@ +/* + * 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.inceptor.converter; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +import com.dtstack.chunjun.constants.ConstantValue; +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.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.ColumnTypeUtil; +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.hadoop.hive.common.type.HiveDate; +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.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +public class InceptorParquetColumnConverter + extends AbstractRowConverter { + private List columnNameList; + private transient Map decimalColInfo; + + public InceptorParquetColumnConverter(List fieldConfList) { + super(fieldConfList.size()); + for (int i = 0; i < fieldConfList.size(); i++) { + String type = fieldConfList.get(i).getType(); + int left = type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL); + int right = type.indexOf(ConstantValue.RIGHT_PARENTHESIS_SYMBOL); + if (left > 0 && right > 0) { + type = type.substring(0, left); + } + toInternalConverters.add( + wrapIntoNullableInternalConverter(createInternalConverter(type))); + toExternalConverters.add( + wrapIntoNullableExternalConverter(createExternalConverter(type), type)); + } + } + + @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 < rowData.getArity(); index++) { + toExternalConverters.get(index).serialize(rowData, index, group); + } + return group; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); + } + + @Override + @SuppressWarnings("unchecked") + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, String type) { + return (rowData, index, group) -> { + if (rowData == null || rowData.isNullAt(index)) { + // do nothing + } else { + serializationConverter.serialize(rowData, index, group); + } + }; + } + + @Override + @SuppressWarnings("all") + protected IDeserializationConverter createInternalConverter(String type) { + switch (type.toUpperCase(Locale.ENGLISH)) { + case "BOOLEAN": + return val -> new BooleanColumn(Boolean.valueOf(val.toString())); + case "TINYINT": + case "SMALLINT": + case "INT": + return val -> new BigDecimalColumn(val.toString()); + case "BIGINT": + return val -> new BigDecimalColumn(val.toString()); + case "FLOAT": + return val -> new BigDecimalColumn(val.toString()); + case "DOUBLE": + return val -> new BigDecimalColumn(val.toString()); + case "DECIMAL": + return val -> new BigDecimalColumn(val.toString()); + case "STRING": + case "VARCHAR": + case "CHAR": + return val -> new StringColumn(val.toString()); + case "TIMESTAMP": + return (IDeserializationConverter) + TimestampColumn::new; + case "DATE": + return (IDeserializationConverter) + val -> new TimestampColumn(DateUtil.getTimestampFromStr(val)); + case "BINARY": + return (IDeserializationConverter) BytesColumn::new; + case "ARRAY": + case "MAP": + case "STRUCT": + case "UNION": + default: + throw new UnsupportedTypeException(type); + } + } + + @Override + protected ISerializationConverter createExternalConverter(String type) { + switch (type.toUpperCase(Locale.ENGLISH)) { + case "BOOLEAN": + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getBoolean(index)); + case "TINYINT": + case "SMALLINT": + case "INT": + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getInt(index)); + case "BIGINT": + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getLong(index)); + case "FLOAT": + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getFloat(index)); + case "DOUBLE": + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getDouble(index)); + case "DECIMAL": + return (rowData, index, group) -> { + ColumnTypeUtil.DecimalInfo decimalInfo = + decimalColInfo.get(columnNameList.get(index)); + HiveDecimal hiveDecimal = + HiveDecimal.create( + rowData.getDecimal( + index, + decimalInfo.getPrecision(), + decimalInfo.getScale()) + .toBigDecimal()); + hiveDecimal = + HiveDecimal.enforcePrecisionScale( + hiveDecimal, + decimalInfo.getPrecision(), + decimalInfo.getScale()); + if (hiveDecimal == null) { + String msg = + String.format( + "The [%s] data data [%s] precision and scale do not match the metadata:decimal(%s, %s)", + index, + decimalInfo.getPrecision(), + decimalInfo.getScale(), + rowData); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + group.add( + columnNameList.get(index), + InceptorUtil.decimalToBinary( + hiveDecimal, + decimalInfo.getPrecision(), + decimalInfo.getScale())); + }; + case "STRING": + case "VARCHAR": + case "CHAR": + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getString(index).toString()); + case "TIMESTAMP": + return (rowData, index, group) -> { + TimestampData timestampData = rowData.getTimestamp(index, 6); + group.add( + columnNameList.get(index), + InceptorUtil.timestampToInt96(timestampData)); + }; + case "DATE": + return (rowData, index, group) -> { + TimestampData timestampData = rowData.getTimestamp(index, 6); + Date date = Date.valueOf(timestampData.toLocalDateTime().toLocalDate()); + group.add( + columnNameList.get(index), + DateWritable.dateToDays(new HiveDate(date.getTime()))); + }; + case "BINARY": + return (rowData, index, group) -> + group.add( + columnNameList.get(index), + Binary.fromReusedByteArray(rowData.getBinary(index))); + case "ARRAY": + case "MAP": + case "STRUCT": + case "UNION": + default: + throw new UnsupportedTypeException(type); + } + } + + public void setColumnNameList(List columnNameList) { + this.columnNameList = columnNameList; + } + + public void setDecimalColInfo(Map decimalColInfo) { + this.decimalColInfo = decimalColInfo; + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorParquetRowConverter.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorParquetRowConverter.java new file mode 100644 index 0000000000..83048d74e0 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorParquetRowConverter.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.inceptor.converter; + +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +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 org.apache.hadoop.hive.common.type.HiveDate; +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; + +public class InceptorParquetRowConverter + extends AbstractRowConverter { + + private List columnNameList; + + public InceptorParquetRowConverter(RowType rowType) { + super(rowType); + 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 Group toExternal(RowData rowData, Group group) throws Exception { + for (int index = 0; index < rowData.getArity(); index++) { + toExternalConverters.get(index).serialize(rowData, index, group); + } + return group; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); + } + + @Override + @SuppressWarnings("unchecked") + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, LogicalType type) { + return (rowData, index, group) -> { + if (rowData == null + || rowData.isNullAt(index) + || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) { + // do nothing + } else { + serializationConverter.serialize(rowData, index, group); + } + }; + } + + @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.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(type); + } + } + + @Override + protected ISerializationConverter createExternalConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return (rowData, index, group) -> {}; + case BOOLEAN: + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getBoolean(index)); + case TINYINT: + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getByte(index)); + case SMALLINT: + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getShort(index)); + case INTEGER: + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getInt(index)); + case BIGINT: + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getLong(index)); + case DATE: + return (rowData, index, group) -> { + Date date = Date.valueOf(LocalDate.ofEpochDay(rowData.getInt(index))); + group.add( + columnNameList.get(index), + DateWritable.dateToDays(new HiveDate(date.getTime()))); + }; + case FLOAT: + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getFloat(index)); + case DOUBLE: + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getDouble(index)); + case CHAR: + case VARCHAR: + return (rowData, index, group) -> + group.add(columnNameList.get(index), rowData.getString(index).toString()); + case DECIMAL: + return (rowData, index, group) -> { + 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); + group.add( + columnNameList.get(index), + InceptorUtil.decimalToBinary(hiveDecimal, precision, scale)); + }; + case BINARY: + case VARBINARY: + return (rowData, index, group) -> + group.add( + columnNameList.get(index), + Binary.fromReusedByteArray(rowData.getBinary(index))); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, group) -> { + TimestampData timestampData = + rowData.getTimestamp(index, ((TimestampType) type).getPrecision()); + group.add( + columnNameList.get(index), + InceptorUtil.timestampToInt96(timestampData)); + }; + 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); + } + } + + public void setColumnNameList(List columnNameList) { + this.columnNameList = columnNameList; + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorTextColumnConvent.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorTextColumnConvent.java new file mode 100644 index 0000000000..2e44bf14f7 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorTextColumnConvent.java @@ -0,0 +1,233 @@ +/* + * 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.inceptor.converter; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.constants.ConstantValue; +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.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 java.sql.Date; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; + +public class InceptorTextColumnConvent + extends AbstractRowConverter { + + private boolean columnIsStarSymbol; + + public InceptorTextColumnConvent(List fieldConfList) { + super(fieldConfList.size()); + columnIsStarSymbol = + fieldConfList.size() == 1 + && ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName()); + + if (!columnIsStarSymbol) { + for (int i = 0; i < fieldConfList.size(); i++) { + String type = fieldConfList.get(i).getType(); + int left = type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL); + int right = type.indexOf(ConstantValue.RIGHT_PARENTHESIS_SYMBOL); + if (left > 0 && right > 0) { + type = type.substring(0, left); + } + toInternalConverters.add( + wrapIntoNullableInternalConverter(createInternalConverter(type))); + toExternalConverters.add( + wrapIntoNullableExternalConverter(createExternalConverter(type), type)); + } + } + } + + @Override + public RowData toInternal(RowData input) throws Exception { + ColumnRowData row = new ColumnRowData(input.getArity()); + if (input instanceof GenericRowData) { + GenericRowData genericRowData = (GenericRowData) input; + if (columnIsStarSymbol) { + for (int i = 0; i < input.getArity(); i++) { + if (genericRowData.getField(i) != null) { + row.addField(new StringColumn(genericRowData.getField(i).toString())); + } else { + row.addField(null); + } + } + } else { + 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 + public String[] toExternal(RowData rowData, String[] output) throws Exception { + for (int index = 0; index < rowData.getArity(); index++) { + toExternalConverters.get(index).serialize(rowData, index, output); + } + return output; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException( + "Inceptor File Connector doesn't support Lookup Table Function."); + } + + @Override + @SuppressWarnings("unchecked") + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, String type) { + return (rowData, index, data) -> { + if (rowData == null || rowData.isNullAt(index)) { + data[index] = null; + } else { + serializationConverter.serialize(rowData, index, data); + } + }; + } + + @Override + protected IDeserializationConverter createInternalConverter(String type) { + switch (type.toUpperCase(Locale.ENGLISH)) { + case "BOOLEAN": + return val -> new BooleanColumn(Boolean.parseBoolean(val.toString())); + case "TINYINT": + case "SMALLINT": + case "INT": + case "BIGINT": + case "FLOAT": + case "DOUBLE": + case "DECIMAL": + return val -> new BigDecimalColumn(val.toString()); + case "STRING": + case "VARCHAR": + case "CHAR": + return val -> new StringColumn(val.toString()); + case "TIMESTAMP": + return (IDeserializationConverter) + val -> { + try { + return new TimestampColumn( + Timestamp.valueOf(val), + DateUtil.getPrecisionFromTimestampStr(val)); + } catch (Exception e) { + return new TimestampColumn(DateUtil.getTimestampFromStr(val), 0); + } + }; + case "DATE": + return (IDeserializationConverter) + val -> { + Timestamp timestamp = DateUtil.getTimestampFromStr(val); + if (timestamp == null) { + return new SqlDateColumn(null); + } else { + return new SqlDateColumn( + Date.valueOf(timestamp.toLocalDateTime().toLocalDate())); + } + }; + case "BINARY": + case "ARRAY": + case "MAP": + case "STRUCT": + case "UNION": + default: + throw new UnsupportedTypeException(type); + } + } + + @Override + protected ISerializationConverter createExternalConverter(String type) { + switch (type.toUpperCase(Locale.ENGLISH)) { + case "BOOLEAN": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getBoolean(index)); + case "TINYINT": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getByte(index)); + case "SMALLINT": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getShort(index)); + case "INT": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getInt(index)); + case "BIGINT": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getLong(index)); + case "FLOAT": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getFloat(index)); + case "DOUBLE": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getDouble(index)); + case "DECIMAL": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getDecimal(index, 38, 18)); + case "STRING": + case "VARCHAR": + case "CHAR": + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getString(index)); + case "TIMESTAMP": + return (rowData, index, data) -> { + AbstractBaseColumn field = ((ColumnRowData) rowData).getField(index); + data[index] = field.asTimestampStr(); + }; + case "DATE": + return (rowData, index, data) -> + data[index] = + String.valueOf( + new Date(rowData.getTimestamp(index, 6).getMillisecond())); + case "BINARY": + return (rowData, index, data) -> + data[index] = Arrays.toString(rowData.getBinary(index)); + case "ARRAY": + case "MAP": + case "STRUCT": + case "UNION": + default: + throw new UnsupportedTypeException(type); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorTextRowConverter.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorTextRowConverter.java new file mode 100644 index 0000000000..52b0641654 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/converter/InceptorTextRowConverter.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.inceptor.converter; + +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; + +public class InceptorTextRowConverter + extends AbstractRowConverter { + + public InceptorTextRowConverter(RowType rowType) { + super(rowType); + 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 String[] toExternal(RowData rowData, String[] data) throws Exception { + for (int index = 0; index < rowData.getArity(); index++) { + toExternalConverters.get(index).serialize(rowData, index, data); + } + return data; + } + + @Override + public RowData toInternalLookup(RowData input) { + throw new ChunJunRuntimeException("HDFS Connector doesn't support Lookup Table Function."); + } + + @Override + @SuppressWarnings("unchecked") + protected ISerializationConverter wrapIntoNullableExternalConverter( + ISerializationConverter serializationConverter, LogicalType type) { + return (rowData, index, data) -> { + if (rowData == null + || rowData.isNullAt(index) + || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) { + data[index] = 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[index] = null; + case BOOLEAN: + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getBoolean(index)); + case TINYINT: + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getByte(index)); + case SMALLINT: + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getShort(index)); + case INTEGER: + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getInt(index)); + case BIGINT: + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getLong(index)); + case DATE: + return (rowData, index, data) -> + data[index] = + String.valueOf( + Date.valueOf(LocalDate.ofEpochDay(rowData.getInt(index)))); + case FLOAT: + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getFloat(index)); + case DOUBLE: + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getDouble(index)); + case CHAR: + case VARCHAR: + return (rowData, index, data) -> + data[index] = String.valueOf(rowData.getString(index)); + case DECIMAL: + return (rowData, index, data) -> + data[index] = + String.valueOf( + rowData.getDecimal( + index, + ((DecimalType) type).getPrecision(), + ((DecimalType) type).getScale())); + case BINARY: + case VARBINARY: + return (rowData, index, data) -> + data[index] = Arrays.toString(rowData.getBinary(index)); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (rowData, index, data) -> + data[index] = + 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-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/dialect/InceptorHdfsDialect.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/dialect/InceptorHdfsDialect.java index 8e45b9a8fd..67f39e9801 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/dialect/InceptorHdfsDialect.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/dialect/InceptorHdfsDialect.java @@ -32,6 +32,7 @@ import org.apache.flink.table.types.logical.RowType; import io.vertx.core.json.JsonArray; +import org.apache.commons.lang3.StringUtils; import java.sql.ResultSet; import java.util.Arrays; @@ -62,22 +63,25 @@ public String getInsertPartitionIntoStatement( .collect(Collectors.joining(", ")); String placeholders = Arrays.stream(fieldNames).map(f -> ":" + f).collect(Collectors.joining(", ")); - return "INSERT INTO " - + buildTableInfoWithSchema(schema, tableName) - + " PARTITION " - + " ( " - + quoteIdentifier(partitionKey) - + "=" - + "'" - + partiitonValue - + "'" - + " ) " - + "(" - + columns - + ")" - + " SELECT " - + placeholders - + " FROM SYSTEM.DUAL"; + StringBuilder stringBuilder = new StringBuilder(10024); + stringBuilder.append("INSERT INTO ").append(buildTableInfoWithSchema(schema, tableName)); + if (StringUtils.isNotBlank(partitionKey)) { + stringBuilder + .append(" PARTITION ( ") + .append(quoteIdentifier(partitionKey)) + .append("= '") + .append(partiitonValue) + .append("' )"); + } + stringBuilder + .append(" (") + .append(columns) + .append(")") + .append(" SELECT ") + .append(placeholders) + .append(" FROM SYSTEM.DUAL"); + + return stringBuilder.toString(); } @Override diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/enums/ECompressType.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/enums/ECompressType.java new file mode 100644 index 0000000000..1693b5d216 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/enums/ECompressType.java @@ -0,0 +1,88 @@ +/* + * 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.inceptor.enums; + +import org.apache.commons.lang.StringUtils; + +public enum ECompressType { + + /** 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 String type; + + private String fileType; + + private String suffix; + + private float deviation; + + ECompressType(String type, String fileType, String suffix, float deviation) { + this.type = type; + this.fileType = fileType; + this.suffix = suffix; + this.deviation = deviation; + } + + public static ECompressType getByTypeAndFileType(String type, String fileType) { + if (StringUtils.isEmpty(type)) { + type = "NONE"; + } + + for (ECompressType value : ECompressType.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-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorOrcInputSplit.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorOrcInputSplit.java new file mode 100644 index 0000000000..7ad61fe928 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorOrcInputSplit.java @@ -0,0 +1,70 @@ +/* + * 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.inceptor.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.Collections; + +public class InceptorOrcInputSplit implements InputSplit { + int splitNumber; + byte[] orcSplitData; + + public InceptorOrcInputSplit(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); + OrcSplit orcSplit = + new OrcSplit( + null, + 0, + 0, + null, + null, + false, + false, + Collections.emptyList(), + Collections.emptyList()); + orcSplit.readFields(dis); + bais.close(); + dis.close(); + return orcSplit; + } + + @Override + public int getSplitNumber() { + return splitNumber; + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorParquetSplit.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorParquetSplit.java new file mode 100644 index 0000000000..a08ec43ed8 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorParquetSplit.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.inceptor.inputSplit; + +import org.apache.flink.core.io.InputSplit; + +import java.util.List; + +public class InceptorParquetSplit implements InputSplit { + + private final int splitNumber; + + private final List paths; + + public InceptorParquetSplit(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-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorTextInputSplit.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorTextInputSplit.java new file mode 100644 index 0000000000..816d696a5b --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/inputSplit/InceptorTextInputSplit.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.inceptor.inputSplit; + +import org.apache.flink.core.io.InputSplit; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileSplit; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +public class InceptorTextInputSplit implements InputSplit { + int splitNumber; + byte[] textSplitData; + + public InceptorTextInputSplit(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((Path) 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-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/lookup/InceptorLruTableFunction.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/lookup/InceptorLruTableFunction.java index 964d1b6ded..7d625085d8 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/lookup/InceptorLruTableFunction.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/lookup/InceptorLruTableFunction.java @@ -41,6 +41,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import static com.dtstack.chunjun.connector.jdbc.options.JdbcLookupOptions.*; + /** * @author dujie @Description * @createTime 2022-01-20 04:50:00 diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/BaseInceptorFileOutputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/BaseInceptorFileOutputFormat.java new file mode 100644 index 0000000000..add78191c3 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/BaseInceptorFileOutputFormat.java @@ -0,0 +1,373 @@ +/* + * 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.inceptor.sink; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.conf.InceptorFileConf; +import com.dtstack.chunjun.connector.inceptor.enums.ECompressType; +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.FileSystemUtil; +import com.dtstack.chunjun.util.PluginUtil; + +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.common.functions.RuntimeContext; + +import com.google.gson.Gson; +import org.apache.commons.collections.CollectionUtils; +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 org.apache.hadoop.security.UserGroupInformation; + +import java.io.File; +import java.io.IOException; +import java.security.PrivilegedAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public abstract class BaseInceptorFileOutputFormat extends BaseFileOutputFormat { + protected InceptorFileConf inceptorFileConf; + protected FileSystem fs; + + protected List fullColumnNameList; + protected List fullColumnTypeList; + protected Configuration conf; + protected transient Map decimalColInfo; + + // 如果key为string类型的值是map 或者 list 会使用gson转为json格式存入 + protected transient Gson gson; + + protected transient UserGroupInformation ugi; + protected boolean openKerberos; + protected ECompressType compressType; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + // 这里休眠一段时间是为了避免reader和writer或者多个任务在同一个taskManager里同时认证kerberos + gson = new Gson(); + openKerberos = FileSystemUtil.isOpenKerberos(inceptorFileConf.getHadoopConfig()); + // 这里休眠一段时间是为了避免reader和writer或者多个任务在同一个taskmanager里同时认证kerberos + if (FileSystemUtil.isOpenKerberos(inceptorFileConf.getHadoopConfig())) { + sleepRandomTime(); + } + + if (openKerberos) { + ugi = + FileSystemUtil.getUGI( + inceptorFileConf.getHadoopConfig(), + inceptorFileConf.getDefaultFs(), + getRuntimeContext().getDistributedCache()); + } else { + String currentUser = FileSystemUtil.getHadoopUser(inceptorFileConf.getHadoopConfig()); + this.ugi = UserGroupInformation.createRemoteUser(currentUser); + } + + super.openInternal(taskNumber, numTasks); + } + + @Override + protected void initVariableFields() { + if (CollectionUtils.isNotEmpty(inceptorFileConf.getFullColumnName())) { + fullColumnNameList = inceptorFileConf.getFullColumnName(); + } else { + fullColumnNameList = + inceptorFileConf.getColumn().stream() + .map(FieldConf::getName) + .collect(Collectors.toList()); + } + + if (CollectionUtils.isNotEmpty(inceptorFileConf.getFullColumnType())) { + fullColumnTypeList = inceptorFileConf.getFullColumnType(); + } else { + fullColumnTypeList = + inceptorFileConf.getColumn().stream() + .map(FieldConf::getType) + .collect(Collectors.toList()); + } + super.initVariableFields(); + } + + @Override + protected void checkOutputDir() { + if (inceptorFileConf.isTransaction()) { + return; + } + Path dir = new Path(tmpPath); + if (fs == null) { + openSource(); + } + + ugi.doAs( + new PrivilegedAction() { + @Override + public Object run() { + try { + 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); + } + return null; + } + }); + } + + @Override + protected void deleteDataDir() { + deleteDirectory(outputFilePath); + } + + @Override + protected void deleteTmpDataDir() { + deleteDirectory(tmpPath); + } + + @Override + protected void openSource() { + DistributedCache distributedCache = getDistributedCache(); + + if (ugi == null) { + try { + openKerberos = FileSystemUtil.isOpenKerberos(inceptorFileConf.getHadoopConfig()); + // 这里休眠一段时间是为了避免reader和writer或者多个任务在同一个taskmanager里同时认证kerberos + if (openKerberos) { + sleepRandomTime(); + } + + if (openKerberos) { + ugi = + FileSystemUtil.getUGI( + inceptorFileConf.getHadoopConfig(), + inceptorFileConf.getDefaultFs(), + distributedCache); + + } else { + String currentUser = + FileSystemUtil.getHadoopUser(inceptorFileConf.getHadoopConfig()); + this.ugi = UserGroupInformation.createRemoteUser(currentUser); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + conf = + FileSystemUtil.getConfiguration( + inceptorFileConf.getHadoopConfig(), inceptorFileConf.getDefaultFs()); + + ugi.doAs( + new PrivilegedAction() { + @Override + public FileSystem run() { + try { + fs = + FileSystemUtil.getFileSystem( + inceptorFileConf.getHadoopConfig(), + inceptorFileConf.getDefaultFs(), + distributedCache); + return null; + } 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 (inceptorFileConf.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 (inceptorFileConf.isTransaction()) { + return; + } + 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 + * + * @return + */ + protected abstract ECompressType getCompressType(); + + protected void deleteDirectory(String path) { + 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)); + } else { + fs.delete(dir, true); + } + } + } catch (IOException e) { + throw new ChunJunRuntimeException("cannot delete directory: " + path, e); + } + } + + public void setInceptorFileConf(InceptorFileConf inceptorFileConf) { + this.inceptorFileConf = inceptorFileConf; + } + + public void setCompressType(ECompressType compressType) { + this.compressType = compressType; + } + + public InceptorFileConf getInceptorFileConf() { + return inceptorFileConf; + } + + private void sleepRandomTime() { + try { + Thread.sleep(5000L + (long) (10000 * Math.random())); + } catch (Exception exception) { + LOG.warn("", exception); + } + } + + protected DistributedCache getDistributedCache() { + RuntimeContext runtimeContext = null; + try { + runtimeContext = getRuntimeContext(); + } catch (IllegalStateException e) { + // ignore + } + DistributedCache distributedCache; + if (runtimeContext == null) { + distributedCache = PluginUtil.createDistributedCacheFromContextClassLoader(); + } else { + distributedCache = runtimeContext.getDistributedCache(); + } + return distributedCache; + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileOrcOutputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileOrcOutputFormat.java new file mode 100644 index 0000000000..6ec0048f71 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileOrcOutputFormat.java @@ -0,0 +1,389 @@ +/* + * 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.inceptor.sink; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorOrcColumnConvent; +import com.dtstack.chunjun.connector.inceptor.enums.ECompressType; +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +import com.dtstack.chunjun.enums.ColumnType; +import com.dtstack.chunjun.security.KerberosUtil; +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.commons.lang3.StringUtils; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Table; +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.inceptor.streaming.HiveStreamingConnection; +import org.apache.inceptor.streaming.StreamingException; +import org.apache.inceptor.streaming.StrictDelimitedInputWriter; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Field; +import java.security.PrivilegedAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.stream.Collectors; + +import static com.dtstack.chunjun.connector.inceptor.util.InceptorDbUtil.KEY_PRINCIPAL; + +public class InceptorFileOrcOutputFormat extends BaseInceptorFileOutputFormat { + + private static ColumnTypeUtil.DecimalInfo ORC_DEFAULT_DECIMAL_INFO = + new ColumnTypeUtil.DecimalInfo( + HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE); + private RecordWriter recordWriter; + private OrcSerde orcSerde; + private StructObjectInspector inspector; + private FileOutputFormat outputFormat; + private JobConf jobConf; + private StrictDelimitedInputWriter wr; + private HiveConf hiveConf; + private HiveMetaStoreClient hiveMetaStoreClient; + private HiveStreamingConnection connection; + + protected List columnNames; + + @Override + protected void initVariableFields() { + columnNames = + inceptorFileConf.getColumn().stream() + .map(FieldConf::getName) + .collect(Collectors.toList()); + + super.initVariableFields(); + } + + @Override + protected void openSource() { + super.openSource(); + if (inceptorFileConf.isTransaction()) { + ugi.doAs( + new PrivilegedAction() { + public Void run() { + try { + hiveConf = new HiveConf(); + hiveConf.addResource(conf); + if (openKerberos) { + setMetaStoreKerberosConf(); + } + wr = + StrictDelimitedInputWriter.newBuilder() + .withFieldDelimiter(',') + .build(); + hiveMetaStoreClient = new HiveMetaStoreClient(hiveConf); + setFullColumns(); + } catch (Exception e) { + throw new RuntimeException("init client failed", e); + } + return null; + } + }); + return; + } + orcSerde = new OrcSerde(); + outputFormat = new org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat(); + jobConf = new JobConf(conf); + + FileOutputFormat.setOutputCompressorClass(jobConf, getOrcCompressType()); + + List fullColTypeList = new ArrayList<>(); + decimalColInfo = new HashMap<>((fullColumnTypeList.size() << 2) / 3); + for (int i = 0; i < fullColumnTypeList.size(); i++) { + String columnType = fullColumnTypeList.get(i); + + if (ColumnTypeUtil.isDecimalType(columnType)) { + ColumnTypeUtil.DecimalInfo decimalInfo = + ColumnTypeUtil.getDecimalInfo(columnType, ORC_DEFAULT_DECIMAL_INFO); + decimalColInfo.put(fullColumnNameList.get(i), decimalInfo); + } + + ColumnType type = ColumnType.getType(columnType); + fullColTypeList.add(InceptorUtil.columnTypeToObjectInspetor(type)); + } + + if (rowConverter instanceof InceptorOrcColumnConvent) { + ((InceptorOrcColumnConvent) rowConverter).setDecimalColInfo(decimalColInfo); + ((InceptorOrcColumnConvent) rowConverter) + .setColumnNameList( + inceptorFileConf.getColumn().stream() + .map(FieldConf::getName) + .collect(Collectors.toList())); + } + + this.inspector = + ObjectInspectorFactory.getStandardStructObjectInspector( + fullColumnNameList, fullColTypeList); + } + + @Override + protected void nextBlock() { + if (inceptorFileConf.isTransaction()) { + return; + } + + 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 (IOException | IllegalAccessException e) { + throw new ChunJunRuntimeException( + InceptorUtil.parseErrorMsg(null, ExceptionUtil.getErrorMessage(e)), e); + } + } + + @Override + protected void writeSingleRecordToFile(RowData rowData) throws WriteRecordException { + + if (inceptorFileConf.isTransaction()) { + String rowString; + try { + Object[] data = new Object[inceptorFileConf.getColumn().size()]; + try { + data = (Object[]) rowConverter.toExternal(rowData, data); + } catch (Exception e) { + String errorMessage = + InceptorUtil.parseErrorMsg( + String.format("writer hdfs error,rowData:{%s}", rowData), + ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(errorMessage, e, -1, rowData); + } + StringBuilder str = new StringBuilder(); + for (int columnIndex = 0; columnIndex < fullColumnNameList.size(); columnIndex++) { + if (columnNames.contains(fullColumnNameList.get(columnIndex))) { + if (data[columnIndex] == null) { + str.append(","); + continue; + } + + str.append(data[columnIndex]).append(","); + continue; + } + str.append(","); + } + rowString = str.toString(); + + } catch (Exception e) { + throw new WriteRecordException(String.format("数据写入hdfs异常,row:{%s}", rowData), e); + } + ugi.doAs( + new PrivilegedAction() { + public Void run() { + try { + if (connection == null) { + initConnection(); + connection.beginTransaction(); + } + connection.write(rowString.getBytes()); + } catch (Exception e) { + throw new RuntimeException("WRITER DATA ERROR", e); + } + return null; + } + }); + rowsOfCurrentBlock++; + lastRow = rowData; + return; + } + + if (recordWriter == null) { + nextBlock(); + } + + Object[] data = new Object[inceptorFileConf.getColumn().size()]; + try { + data = (Object[]) rowConverter.toExternal(rowData, data); + } catch (Exception e) { + String errorMessage = + InceptorUtil.parseErrorMsg( + String.format("writer hdfs error,rowData:{%s}", rowData), + ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(errorMessage, e, -1, rowData); + } + + try { + this.recordWriter.write( + NullWritable.get(), this.orcSerde.serialize(data, this.inspector)); + rowsOfCurrentBlock++; + lastRow = rowData; + } catch (IOException e) { + throw new WriteRecordException( + String.format("Data writing to hdfs is abnormal,rowData:{%s}", rowData), e); + } + } + + @Override + protected void flushDataInternal() { + LOG.info( + "Close current orc record writer, write data size:[{}]", + bytesWriteCounter.getLocalValue()); + if (inceptorFileConf.isTransaction()) { + if (connection == null) { + return; + } + ugi.doAs( + new PrivilegedAction() { + public Void run() { + try { + connection.commitTransaction(); + connection.close(); + connection = null; + } catch (Exception e) { + LOG.error("flush data error", e); + } + return null; + } + }); + return; + } + try { + if (recordWriter != null) { + recordWriter.close(Reporter.NULL); + recordWriter = null; + } + } catch (IOException e) { + throw new ChunJunRuntimeException( + InceptorUtil.parseErrorMsg( + "error to flush stream.", ExceptionUtil.getErrorMessage(e)), + e); + } + } + + @Override + protected ECompressType getCompressType() { + return ECompressType.getByTypeAndFileType(inceptorFileConf.getCompress(), "ORC"); + } + + private void setMetaStoreKerberosConf() { + String keytabFileName = + KerberosUtil.getPrincipalFileName(inceptorFileConf.getHadoopConfig()); + keytabFileName = + KerberosUtil.loadFile( + inceptorFileConf.getHadoopConfig(), keytabFileName, getDistributedCache()); + String principal = + inceptorFileConf.getHadoopConfig().get(KEY_PRINCIPAL) == null + ? null + : inceptorFileConf.getHadoopConfig().get(KEY_PRINCIPAL).toString(); + 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, "true"); + } + + private void setFullColumns() throws Exception { + fullColumnNameList.clear(); + fullColumnTypeList.clear(); + Table inceptorTable = + hiveMetaStoreClient.getTable( + inceptorFileConf.getSchema(), inceptorFileConf.getTable()); + List cols = inceptorTable.getSd().getCols(); + if (cols != null && cols.size() > 0) { + cols.forEach( + fieldSchema -> { + fullColumnNameList.add(fieldSchema.getName()); + fullColumnTypeList.add(fieldSchema.getType()); + }); + } + } + + private Class getOrcCompressType() { + ECompressType compressType = + ECompressType.getByTypeAndFileType(inceptorFileConf.getCompress(), "orc"); + if (ECompressType.ORC_SNAPPY.equals(compressType)) { + return SnappyCodec.class; + } else if (ECompressType.ORC_BZIP.equals(compressType)) { + return BZip2Codec.class; + } else if (ECompressType.ORC_GZIP.equals(compressType)) { + return GzipCodec.class; + } else if (ECompressType.ORC_LZ4.equals(compressType)) { + return Lz4Codec.class; + } else { + return DefaultCodec.class; + } + } + + /** + * 数据源开启kerberos时 如果这里不通过反射对 writerOptions 赋值fs,则在recordWriter.writer时 会初始化一个fs 此fs不在ugi里获取的 + * 导致开启了kerberos的数据源在checkpoint时进行 recordWriter.close() 操作,会出现kerberos认证错误 + * + * @throws IllegalAccessException + */ + private void setFs() throws IllegalAccessException { + 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); + } + + private void initConnection() throws StreamingException { + HiveStreamingConnection.Builder builder = + HiveStreamingConnection.newBuilder() + .withDatabase(inceptorFileConf.getSchema()) + .withTable(inceptorFileConf.getTable()) + .withAgentInfo("UT_" + Thread.currentThread().getName()) + .withTransactionBatchSize(1) + .withRecordWriter(wr) + .withHiveConf(hiveConf) + .withClient(hiveMetaStoreClient); + if (StringUtils.isNotEmpty(inceptorFileConf.getPartitionName())) { + List partitions = Arrays.asList(inceptorFileConf.getPartitionName()); + builder.withStaticPartitionValues(partitions); + } + connection = builder.connect(); + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileOutputFormatBuilder.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileOutputFormatBuilder.java new file mode 100644 index 0000000000..5c4d6fb01e --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileOutputFormatBuilder.java @@ -0,0 +1,74 @@ +/* + * 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.inceptor.sink; + +import com.dtstack.chunjun.connector.inceptor.conf.InceptorFileConf; +import com.dtstack.chunjun.connector.inceptor.enums.ECompressType; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.sink.format.FileOutputFormatBuilder; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Locale; + +public class InceptorFileOutputFormatBuilder extends FileOutputFormatBuilder { + private final BaseInceptorFileOutputFormat format; + + public InceptorFileOutputFormatBuilder(String fileType) { + switch (fileType.toUpperCase(Locale.ENGLISH)) { + case "ORC": + format = new InceptorFileOrcOutputFormat(); + break; + case "PARQUET": + format = new InceptorFileParquetOutputFormat(); + break; + default: + format = new InceptorFileTextOutputFormat(); + } + super.setFormat(format); + } + + public void setInceptorConf(InceptorFileConf inceptorFileConf) { + super.setBaseFileConf(inceptorFileConf); + format.setInceptorFileConf(inceptorFileConf); + } + + public void setCompressType(ECompressType compressType) { + format.setCompressType(compressType); + } + + @Override + protected void checkFormat() { + StringBuilder errorMessage = new StringBuilder(256); + InceptorFileConf hdfsConf = format.getInceptorFileConf(); + if (StringUtils.isBlank(hdfsConf.getPath())) { + errorMessage.append("No path supplied. \n"); + } + + if (StringUtils.isBlank(hdfsConf.getDefaultFs())) { + errorMessage.append("No defaultFS supplied. \n"); + } else if (!hdfsConf.getDefaultFs().startsWith(ConstantValue.PROTOCOL_HDFS)) { + errorMessage.append("defaultFS should start with hdfs:// \n"); + } + if (StringUtils.isNotBlank(errorMessage)) { + throw new ChunJunRuntimeException(errorMessage.toString()); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileParquetOutputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileParquetOutputFormat.java new file mode 100644 index 0000000000..ba5b6b4006 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileParquetOutputFormat.java @@ -0,0 +1,277 @@ +/* + * 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.inceptor.sink; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorParquetColumnConverter; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorParquetRowConverter; +import com.dtstack.chunjun.connector.inceptor.enums.ECompressType; +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +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 org.apache.flink.table.data.RowData; + +import org.apache.hadoop.fs.Path; +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.List; +import java.util.stream.Collectors; + +public class InceptorFileParquetOutputFormat extends BaseInceptorFileOutputFormat { + private SimpleGroupFactory groupFactory; + + private ParquetWriter writer; + + private MessageType schema; + + private static ColumnTypeUtil.DecimalInfo PARQUET_DEFAULT_DECIMAL_INFO = + new ColumnTypeUtil.DecimalInfo(10, 0); + + @Override + protected void openSource() { + super.openSource(); + + schema = buildSchema(); + GroupWriteSupport.setSchema(schema, conf); + groupFactory = new SimpleGroupFactory(schema); + List columnNameList = + inceptorFileConf.getColumn().stream() + .map(FieldConf::getName) + .collect(Collectors.toList()); + if (rowConverter instanceof InceptorParquetColumnConverter) { + ((InceptorParquetColumnConverter) rowConverter).setColumnNameList(columnNameList); + ((InceptorParquetColumnConverter) rowConverter).setDecimalColInfo(decimalColInfo); + } else if (rowConverter instanceof InceptorParquetRowConverter) { + ((InceptorParquetRowConverter) rowConverter).setColumnNameList(columnNameList); + } + } + + @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(inceptorFileConf.isEnableDictionary()) + .withRowGroupSize(inceptorFileConf.getRowGroupSize()); + + ugi.doAs( + (PrivilegedAction) + () -> { + try { + writer = builder.build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return null; + }); + currentFileIndex++; + } catch (Exception e) { + throw new ChunJunRuntimeException( + InceptorUtil.parseErrorMsg(null, ExceptionUtil.getErrorMessage(e)), e); + } + } + + @Override + public ECompressType getCompressType() { + return ECompressType.getByTypeAndFileType(inceptorFileConf.getCompress(), "PARQUET"); + } + + @Override + @SuppressWarnings("unchecked") + public void writeSingleRecordToFile(RowData rowData) throws WriteRecordException { + if (writer == null) { + nextBlock(); + } + + Group group = groupFactory.newGroup(); + try { + group = (Group) rowConverter.toExternal(rowData, group); + } catch (Exception e) { + String errorMessage = + InceptorUtil.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 + 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( + InceptorUtil.parseErrorMsg( + "error to flush stream.", ExceptionUtil.getErrorMessage(e)), + 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(); + } + } + + 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 = + ColumnType.fromString(fullColumnTypeList.get(i).toLowerCase()) + .name() + .toLowerCase(); + 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( + InceptorUtil.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-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileSinkFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileSinkFactory.java new file mode 100644 index 0000000000..62c9ce9210 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileSinkFactory.java @@ -0,0 +1,90 @@ +/* + * 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.inceptor.sink; + +import com.dtstack.chunjun.conf.SyncConf; +import com.dtstack.chunjun.connector.inceptor.conf.InceptorFileConf; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorHdfsRawTypeConverter; +import com.dtstack.chunjun.connector.inceptor.enums.ECompressType; +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +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 java.util.Map; + +public class InceptorFileSinkFactory extends SinkFactory { + private InceptorFileConf inceptorFileConf; + + public InceptorFileSinkFactory(SyncConf syncConf) { + super(syncConf); + this.inceptorFileConf = + GsonUtil.GSON.fromJson( + GsonUtil.GSON.toJson(syncConf.getWriter().getParameter()), + InceptorFileConf.class); + inceptorFileConf.setColumn(syncConf.getWriter().getFieldList()); + super.initCommonConf(inceptorFileConf); + Map parameter = syncConf.getWriter().getParameter(); + if (null != parameter.get("defaultFS")) { + inceptorFileConf.setDefaultFs(parameter.get("defaultFS").toString()); + } + + if (null != parameter.get("isTransaction")) { + inceptorFileConf.setTransaction((Boolean) parameter.get("isTransaction")); + } + + if (parameter.get("fieldDelimiter") == null + || parameter.get("fieldDelimiter").toString().length() == 0) { + inceptorFileConf.setFieldDelimiter("\001"); + } else { + inceptorFileConf.setFieldDelimiter( + com.dtstack.chunjun.util.StringUtil.convertRegularExpr( + parameter.get("fieldDelimiter").toString())); + } + } + + @Override + public DataStreamSink createSink(DataStream dataSet) { + InceptorFileOutputFormatBuilder builder = + new InceptorFileOutputFormatBuilder(inceptorFileConf.getFileType()); + builder.setInceptorConf(inceptorFileConf); + builder.setCompressType( + ECompressType.getByTypeAndFileType( + inceptorFileConf.getCompress(), inceptorFileConf.getFileType())); + AbstractRowConverter rowConverter = + InceptorUtil.createRowConverter( + useAbstractBaseColumn, + inceptorFileConf.getFileType(), + inceptorFileConf.getColumn(), + getRawTypeConverter()); + + builder.setRowConverter(rowConverter); + return createOutput(dataSet, builder.finish()); + } + + @Override + public RawTypeConverter getRawTypeConverter() { + return InceptorHdfsRawTypeConverter::apply; + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileTextOutputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileTextOutputFormat.java new file mode 100644 index 0000000000..7ae952524e --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileTextOutputFormat.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.inceptor.sink; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.enums.ECompressType; +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +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; + +public class InceptorFileTextOutputFormat extends BaseInceptorFileOutputFormat { + + private static final int NEWLINE = 10; + private transient OutputStream stream; + + private static final int BUFFER_SIZE = 1000; + + @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 + protected ECompressType getCompressType() { + return ECompressType.getByTypeAndFileType(inceptorFileConf.getCompress(), "TEXT"); + } + + @Override + protected void writeSingleRecordToFile(RowData rowData) throws WriteRecordException { + if (stream == null) { + nextBlock(); + } + String[] data = new String[inceptorFileConf.getColumn().size()]; + try { + data = (String[]) rowConverter.toExternal(rowData, data); + } catch (Exception e) { + throw new WriteRecordException("can't parse rowData", e, -1, rowData); + } + + String[] result = new String[fullColumnNameList.size()]; + for (int i = 0; i < inceptorFileConf.getColumn().size(); i++) { + FieldConf fieldConf = inceptorFileConf.getColumn().get(i); + result[fieldConf.getIndex()] = data[i]; + } + String line = String.join(inceptorFileConf.getFieldDelimiter(), result); + + try { + byte[] bytes = line.getBytes(inceptorFileConf.getEncoding()); + this.stream.write(bytes); + this.stream.write(NEWLINE); + rowsOfCurrentBlock++; + lastRow = rowData; + } catch (IOException e) { + String errorMessage = + InceptorUtil.parseErrorMsg( + String.format("writer hdfs error,rowData:{%s}", rowData), + ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(errorMessage, e, -1, rowData); + } + } + + @Override + protected 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( + InceptorUtil.parseErrorMsg( + "error to flush stream.", ExceptionUtil.getErrorMessage(e)), + e); + } + } + + @Override + protected void nextBlock() { + super.nextBlock(); + + if (stream != null) { + return; + } + + try { + String currentBlockTmpPath = tmpPath + File.separatorChar + currentFileName; + Path p = new Path(currentBlockTmpPath); + + if (ECompressType.TEXT_NONE.equals(compressType)) { + stream = fs.create(p); + } else { + p = new Path(currentBlockTmpPath); + if (compressType == ECompressType.TEXT_GZIP) { + stream = new GzipCompressorOutputStream(fs.create(p)); + } else if (compressType == ECompressType.TEXT_BZIP2) { + stream = new BZip2CompressorOutputStream(fs.create(p)); + } + } + currentFileIndex++; + LOG.info("subtask:[{}] create block file:{}", taskNumber, currentBlockTmpPath); + } catch (IOException e) { + throw new ChunJunRuntimeException( + InceptorUtil.parseErrorMsg(null, ExceptionUtil.getErrorMessage(e)), e); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorHdfsOutputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorHdfsOutputFormat.java index 15b5ab3d03..f3d580ebdb 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorHdfsOutputFormat.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorHdfsOutputFormat.java @@ -72,7 +72,7 @@ public class InceptorHdfsOutputFormat extends JdbcOutputFormat { private String currentPartition; // 是否是事务表 - private boolean isTransactionTable; + private Boolean isTransactionTable; // 当前事务是否已开启 private volatile boolean transactionStart; @@ -82,16 +82,21 @@ public class InceptorHdfsOutputFormat extends JdbcOutputFormat { @Override public void open(int taskNumber, int numTasks) throws IOException { super.open(taskNumber, numTasks); - if (isTransactionTable) { - super.checkpointMode = CheckpointingMode.EXACTLY_ONCE; - super.semantic = Semantic.EXACTLY_ONCE; - } } @Override protected void openInternal(int taskNumber, int numTasks) { try { - partitionFormat = getPartitionFormat(); + // when isTransactionTable is null, the tash is sql + if (isTransactionTable == null) { + partitionFormat = getPartitionFormat(); + } + + // batch and transactionTable + if (isTransactionTable != null && isTransactionTable && this.batchSize == 1) { + this.batchSize = 1024; + } + dbConn = getConnection(); statement = dbConn.createStatement(); // use database @@ -101,7 +106,15 @@ protected void openInternal(int taskNumber, int numTasks) { } initColumnList(); switchNextPartiiton(new Date()); - isTransactionTable = isTransactionTable(); + + // sql + if (isTransactionTable == null) { + isTransactionTable = isTransactionTable(); + if (isTransactionTable) { + super.checkpointMode = CheckpointingMode.EXACTLY_ONCE; + super.semantic = Semantic.EXACTLY_ONCE; + } + } LOG.info("subTask[{}}] wait finished", taskNumber); } catch (SQLException throwables) { throw new IllegalArgumentException("open() failed.", throwables); @@ -121,6 +134,11 @@ protected void writeSingleRecordInternal(RowData rowData) throws WriteRecordExce transactionStart = true; } stmtProxy.writeSingleRecordInternal(rowData); + + if (Semantic.EXACTLY_ONCE != semantic) { + doCommit(); + } + } catch (Exception e) { processWriteException(e, index, rowData); } @@ -173,6 +191,8 @@ protected void writeMultipleRecordsInternal() throws Exception { // 开启了cp,但是并没有使用2pc方式让下游数据可见 if (Semantic.EXACTLY_ONCE == semantic) { rowsOfCurrentTransaction += rows.size(); + } else { + doCommit(); } } catch (Exception e) { LOG.warn( @@ -186,8 +206,7 @@ protected void writeMultipleRecordsInternal() throws Exception { } } - @Override - public void commit(long checkpointId) throws Exception { + public void doCommit() throws SQLException { try { if (transactionStart) { Statement statement = dbConn.createStatement(); @@ -265,16 +284,20 @@ private SimpleDateFormat getPartitionFormat() { } private void switchNextPartiiton(Date currentData) throws SQLException { - String newPartition = partitionFormat.format(currentData); - if (StringUtils.isBlank(currentPartition) || !currentPartition.equals(newPartition)) { - LOG.info( - "switch old partition {} to new partition {}", currentPartition, newPartition); - if (stmtProxy != null) { - stmtProxy.close(); + if (partitionFormat != null) { + String newPartition = partitionFormat.format(currentData); + if (StringUtils.isBlank(currentPartition) || !currentPartition.equals(newPartition)) { + LOG.info( + "switch old partition {} to new partition {}", + currentPartition, + newPartition); + if (stmtProxy != null) { + stmtProxy.close(); + } + currentPartition = newPartition; } - currentPartition = newPartition; - buildStmtProxy(); } + buildStmtProxy(); } public void setJdbcConf(JdbcConf jdbcConf) { @@ -328,4 +351,8 @@ private boolean isTransactionTable() throws SQLException { } })); } + + public void setTransactionTable(Boolean transactionTable) { + isTransactionTable = transactionTable; + } } diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorHdfsOutputFormatBuilder.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorHdfsOutputFormatBuilder.java index 973c6098d7..6ab0a9ed45 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorHdfsOutputFormatBuilder.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorHdfsOutputFormatBuilder.java @@ -35,6 +35,10 @@ public void setRowConverter(AbstractRowConverter rowConverter) { format.setRowConverter(rowConverter); } + public void setTransactionTable(boolean transactionTable) { + ((InceptorHdfsOutputFormat) format).setTransactionTable(transactionTable); + } + @Override protected void checkFormat() { JdbcConf jdbcConf = format.getJdbcConf(); diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorJdbcSinkFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorJdbcSinkFactory.java new file mode 100644 index 0000000000..503d0dad74 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorJdbcSinkFactory.java @@ -0,0 +1,79 @@ +/* + * 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.inceptor.sink; + +import com.dtstack.chunjun.conf.SyncConf; +import com.dtstack.chunjun.connector.inceptor.conf.InceptorConf; +import com.dtstack.chunjun.connector.inceptor.dialect.InceptorDialect; +import com.dtstack.chunjun.connector.inceptor.util.InceptorDbUtil; +import com.dtstack.chunjun.connector.jdbc.conf.JdbcConf; +import com.dtstack.chunjun.connector.jdbc.sink.JdbcOutputFormatBuilder; +import com.dtstack.chunjun.connector.jdbc.sink.JdbcSinkFactory; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.util.TableUtil; + +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.flink.table.types.logical.RowType; + +public class InceptorJdbcSinkFactory extends JdbcSinkFactory { + + InceptorDialect inceptorDialect; + + public InceptorJdbcSinkFactory(SyncConf syncConf) { + super(syncConf, null); + inceptorDialect = InceptorDbUtil.getDialectWithDriverType(jdbcConf); + jdbcConf.setJdbcUrl(inceptorDialect.appendJdbcTransactionType(jdbcConf.getJdbcUrl())); + super.jdbcDialect = inceptorDialect; + } + + @Override + public DataStreamSink createSink(DataStream dataSet) { + JdbcOutputFormatBuilder builder = getBuilder(); + builder.setJdbcConf(jdbcConf); + builder.setJdbcDialect(jdbcDialect); + + AbstractRowConverter rowConverter = null; + // 同步任务使用transform + if (!useAbstractBaseColumn) { + final RowType rowType = + TableUtil.createRowType(jdbcConf.getColumn(), getRawTypeConverter()); + rowConverter = jdbcDialect.getRowConverter(rowType); + } + builder.setRowConverter(rowConverter); + if (builder instanceof InceptorHdfsOutputFormatBuilder) { + // InceptorHdfsOutputFormatBuilder 只有实时任务或者离线任务的事务表才会调用 所以此处设置为true,其余的orc text + // parquet通过文件方式写入 + ((InceptorHdfsOutputFormatBuilder) builder).setTransactionTable(true); + } + + return createOutput(dataSet, builder.finish()); + } + + @Override + protected Class getConfClass() { + return InceptorConf.class; + } + + @Override + protected JdbcOutputFormatBuilder getBuilder() { + return ((InceptorDialect) jdbcDialect).getOutputFormatBuilder(); + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorSinkFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorSinkFactory.java index 6bcea00a9c..511de72eb1 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorSinkFactory.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorSinkFactory.java @@ -19,32 +19,40 @@ package com.dtstack.chunjun.connector.inceptor.sink; import com.dtstack.chunjun.conf.SyncConf; -import com.dtstack.chunjun.connector.inceptor.conf.InceptorConf; -import com.dtstack.chunjun.connector.inceptor.dialect.InceptorDialect; -import com.dtstack.chunjun.connector.inceptor.util.InceptorDbUtil; -import com.dtstack.chunjun.connector.jdbc.conf.JdbcConf; -import com.dtstack.chunjun.connector.jdbc.sink.JdbcOutputFormatBuilder; -import com.dtstack.chunjun.connector.jdbc.sink.JdbcSinkFactory; +import com.dtstack.chunjun.converter.RawTypeConverter; +import com.dtstack.chunjun.sink.SinkFactory; + +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.MapUtils; /** @author liuliu 2022/2/24 */ -public class InceptorSinkFactory extends JdbcSinkFactory { +public class InceptorSinkFactory extends SinkFactory { - InceptorDialect inceptorDialect; + SinkFactory sinkFactory; public InceptorSinkFactory(SyncConf syncConf) { - super(syncConf, null); - inceptorDialect = InceptorDbUtil.getDialectWithDriverType(jdbcConf); - jdbcConf.setJdbcUrl(inceptorDialect.appendJdbcTransactionType(jdbcConf.getJdbcUrl())); - super.jdbcDialect = inceptorDialect; + super(syncConf); + boolean useJdbc = !syncConf.getWriter().getParameter().containsKey("path"); + boolean transaction = + MapUtils.getBoolean(syncConf.getWriter().getParameter(), "isTransaction", false); + + if (useJdbc || transaction) { + this.sinkFactory = new InceptorJdbcSinkFactory(syncConf); + } else { + this.sinkFactory = new InceptorFileSinkFactory(syncConf); + } } @Override - protected Class getConfClass() { - return InceptorConf.class; + public DataStreamSink createSink(DataStream dataSet) { + return sinkFactory.createSink(dataSet); } @Override - protected JdbcOutputFormatBuilder getBuilder() { - return ((InceptorDialect) jdbcDialect).getOutputFormatBuilder(); + public RawTypeConverter getRawTypeConverter() { + return sinkFactory.getRawTypeConverter(); } } diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/BaseInceptorFileInputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/BaseInceptorFileInputFormat.java new file mode 100644 index 0000000000..25c793b75d --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/BaseInceptorFileInputFormat.java @@ -0,0 +1,170 @@ +/* + * 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.inceptor.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.conf.InceptorFileConf; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.source.format.BaseRichInputFormat; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.util.FileSystemUtil; +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.File; +import java.io.IOException; +import java.security.PrivilegedAction; +import java.util.HashMap; +import java.util.Map; + +public abstract class BaseInceptorFileInputFormat extends BaseRichInputFormat { + + protected InceptorFileConf inceptorFileConf; + + protected boolean openKerberos; + protected transient UserGroupInformation ugi; + + protected transient JobConf jobConf; + protected transient RecordReader recordReader; + protected transient org.apache.hadoop.mapred.InputFormat inputFormat; + + protected Object key; + protected Object value; + + @Override + protected InputSplit[] createInputSplitsInternal(int minNumSplits) throws Exception { + initUgi(); + LOG.info("user:{}, ", ugi.getShortUserName()); + return ugi.doAs( + (PrivilegedAction) + () -> { + try { + return createInceptorSplit(minNumSplits); + } catch (Exception e) { + throw new ChunJunRuntimeException("error to create hdfs splits", e); + } + }); + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + initHadoopJobConf(); + this.inputFormat = createInputFormat(); + openKerberos = FileSystemUtil.isOpenKerberos(inceptorFileConf.getHadoopConfig()); + if (openKerberos) { + ugi = + FileSystemUtil.getUGI( + inceptorFileConf.getHadoopConfig(), + inceptorFileConf.getDefaultFs(), + getRuntimeContext().getDistributedCache()); + } else { + String currentUser = FileSystemUtil.getHadoopUser(inceptorFileConf.getHadoopConfig()); + this.ugi = UserGroupInformation.createRemoteUser(currentUser); + } + } + + @Override + @SuppressWarnings("unchecked") + public boolean reachedEnd() throws IOException { + return !recordReader.next(key, value); + } + + @Override + public void closeInternal() throws IOException { + if (recordReader != null) { + recordReader.close(); + } + } + /** + * create hdfs data splits + * + * @param minNumSplits + * @return + * @throws IOException + */ + public abstract InputSplit[] createInceptorSplit(int minNumSplits) throws IOException; + + /** + * create hdfs inputFormat + * + * @return org.apache.hadoop.mapred.InputFormat + */ + public abstract org.apache.hadoop.mapred.InputFormat createInputFormat(); + + public void setInceptorFileConf(InceptorFileConf inceptorFileConf) { + this.inceptorFileConf = inceptorFileConf; + } + + protected void initUgi() throws IOException { + this.openKerberos = FileSystemUtil.isOpenKerberos(inceptorFileConf.getHadoopConfig()); + if (openKerberos) { + DistributedCache distributedCache = + PluginUtil.createDistributedCacheFromContextClassLoader(); + this.ugi = + FileSystemUtil.getUGI( + inceptorFileConf.getHadoopConfig(), + inceptorFileConf.getDefaultFs(), + distributedCache); + } else { + String currentUser = FileSystemUtil.getHadoopUser(inceptorFileConf.getHadoopConfig()); + this.ugi = UserGroupInformation.createRemoteUser(currentUser); + } + } + + /** + * Get current partition information from hdfs path + * + * @param path hdfs path + */ + public void findCurrentPartition(Path path) { + Map map = new HashMap<>(16); + String pathStr = path.getParent().toString(); + int index; + while ((index = pathStr.lastIndexOf(ConstantValue.EQUAL_SYMBOL)) > 0) { + int i = pathStr.lastIndexOf(File.separator); + String name = pathStr.substring(i + 1, index); + String value = pathStr.substring(index + 1); + map.put(name, value); + pathStr = pathStr.substring(0, i); + } + + for (FieldConf fieldConf : inceptorFileConf.getColumn()) { + if (fieldConf.getPart()) { + fieldConf.setValue(map.get(fieldConf.getName())); + } + } + } + + /** init Hadoop Job Config */ + protected void initHadoopJobConf() { + jobConf = + FileSystemUtil.getJobConf( + inceptorFileConf.getHadoopConfig(), inceptorFileConf.getDefaultFs()); + jobConf.set(InceptorPathFilter.KEY_REGEX, inceptorFileConf.getFilterRegex()); + FileSystemUtil.setHadoopUserName(jobConf); + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/HdfsPathFilter.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/HdfsPathFilter.java new file mode 100644 index 0000000000..bf8a5d73d2 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/HdfsPathFilter.java @@ -0,0 +1,73 @@ +/* + * 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.inceptor.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-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileInputFormatBuilder.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileInputFormatBuilder.java new file mode 100644 index 0000000000..efb6bfa081 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileInputFormatBuilder.java @@ -0,0 +1,49 @@ +/* + * 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.inceptor.source; + +import com.dtstack.chunjun.connector.inceptor.conf.InceptorFileConf; +import com.dtstack.chunjun.source.format.BaseRichInputFormatBuilder; + +public class InceptorFileInputFormatBuilder extends BaseRichInputFormatBuilder { + + public static InceptorFileInputFormatBuilder newBuild(String type) { + BaseInceptorFileInputFormat format; + if ("ORC".equalsIgnoreCase(type)) { + format = new InceptorOrcInputFormat(); + } else if ("PARQUET".equalsIgnoreCase(type)) { + format = new InceptorParquetInputFormat(); + } else { + format = new InceptorTextInputFormat(); + } + return new InceptorFileInputFormatBuilder(format); + } + + private InceptorFileInputFormatBuilder(BaseInceptorFileInputFormat format) { + super(format); + } + + @Override + protected void checkFormat() {} + + public void setHdfsConf(InceptorFileConf inceptorConf) { + super.setConfig(inceptorConf); + ((BaseInceptorFileInputFormat) format).setInceptorFileConf(inceptorConf); + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileSourceFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileSourceFactory.java new file mode 100644 index 0000000000..5f4d00b333 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileSourceFactory.java @@ -0,0 +1,89 @@ +/* + * 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.inceptor.source; + +import com.dtstack.chunjun.conf.SyncConf; +import com.dtstack.chunjun.connector.inceptor.conf.InceptorFileConf; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorHdfsRawTypeConverter; +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +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; + +import java.util.Map; + +public class InceptorFileSourceFactory extends SourceFactory { + + private InceptorFileConf inceptorFileConf; + + protected InceptorFileSourceFactory(SyncConf syncConf, StreamExecutionEnvironment env) { + super(syncConf, env); + this.inceptorFileConf = + GsonUtil.GSON.fromJson( + GsonUtil.GSON.toJson(syncConf.getReader().getParameter()), + InceptorFileConf.class); + inceptorFileConf.setColumn(syncConf.getReader().getFieldList()); + super.initCommonConf(inceptorFileConf); + + Map parameter = syncConf.getReader().getParameter(); + if (null != parameter.get("defaultFS")) { + inceptorFileConf.setDefaultFs(parameter.get("defaultFS").toString()); + } + + if (null != parameter.get("isTransaction")) { + inceptorFileConf.setTransaction((Boolean) parameter.get("isTransaction")); + } + + if (parameter.get("fieldDelimiter") == null + || parameter.get("fieldDelimiter").toString().length() == 0) { + inceptorFileConf.setFieldDelimiter("\001"); + } else { + inceptorFileConf.setFieldDelimiter( + com.dtstack.chunjun.util.StringUtil.convertRegularExpr( + parameter.get("fieldDelimiter").toString())); + } + } + + @Override + public RawTypeConverter getRawTypeConverter() { + return InceptorHdfsRawTypeConverter::apply; + } + + @Override + public DataStream createSource() { + InceptorFileInputFormatBuilder builder = + InceptorFileInputFormatBuilder.newBuild(inceptorFileConf.getFileType()); + + builder.setHdfsConf(inceptorFileConf); + AbstractRowConverter rowConverter = + InceptorUtil.createRowConverter( + useAbstractBaseColumn, + inceptorFileConf.getFileType(), + inceptorFileConf.getColumn(), + getRawTypeConverter()); + + builder.setRowConverter(rowConverter); + return createInput(builder.finish()); + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorJdbcSourceFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorJdbcSourceFactory.java new file mode 100644 index 0000000000..129e2eb6a0 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorJdbcSourceFactory.java @@ -0,0 +1,50 @@ +/* + * 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.inceptor.source; + +import com.dtstack.chunjun.conf.SyncConf; +import com.dtstack.chunjun.connector.inceptor.conf.InceptorConf; +import com.dtstack.chunjun.connector.inceptor.dialect.InceptorDialect; +import com.dtstack.chunjun.connector.inceptor.util.InceptorDbUtil; +import com.dtstack.chunjun.connector.jdbc.conf.JdbcConf; +import com.dtstack.chunjun.connector.jdbc.source.JdbcInputFormatBuilder; +import com.dtstack.chunjun.connector.jdbc.source.JdbcSourceFactory; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +public class InceptorJdbcSourceFactory extends JdbcSourceFactory { + InceptorDialect inceptorDialect; + + public InceptorJdbcSourceFactory(SyncConf syncConf, StreamExecutionEnvironment env) { + super(syncConf, env, null); + this.inceptorDialect = InceptorDbUtil.getDialectWithDriverType(jdbcConf); + jdbcConf.setJdbcUrl(inceptorDialect.appendJdbcTransactionType(jdbcConf.getJdbcUrl())); + super.jdbcDialect = inceptorDialect; + } + + @Override + protected Class getConfClass() { + return InceptorConf.class; + } + + @Override + protected JdbcInputFormatBuilder getBuilder() { + return inceptorDialect.getInputFormatBuilder(); + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorOrcInputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorOrcInputFormat.java new file mode 100644 index 0000000000..f9589ea06d --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorOrcInputFormat.java @@ -0,0 +1,408 @@ +/* + * 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.inceptor.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.inputSplit.InceptorOrcInputSplit; +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.throwable.ReadRecordException; +import com.dtstack.chunjun.util.ExceptionUtil; +import com.dtstack.chunjun.util.FileSystemUtil; + +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.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +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; + +public class InceptorOrcInputFormat extends BaseInceptorFileInputFormat { + + protected transient FileSystem fs; + private final AtomicBoolean isInit = new AtomicBoolean(false); + private transient String[] fullColNames; + + private transient StructObjectInspector inspector; + + private transient List fields; + + @Override + public InceptorOrcInputSplit[] createInceptorSplit(int minNumSplits) throws IOException { + super.initHadoopJobConf(); + String path; + if (org.apache.commons.lang3.StringUtils.isNotBlank(inceptorFileConf.getFileName())) { + // 兼容平台逻辑 + path = + inceptorFileConf.getPath() + + ConstantValue.SINGLE_SLASH_SYMBOL + + inceptorFileConf.getFileName(); + } else { + path = inceptorFileConf.getPath(); + } + org.apache.hadoop.mapred.FileInputFormat.setInputPaths(jobConf, path); + org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter( + jobConf, InceptorPathFilter.class); + org.apache.hadoop.mapred.InputSplit[] splits = + new OrcInputFormat().getSplits(jobConf, 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; + // 49B file is empty + if (orcSplit.getLength() > 49) { + list.add(new InceptorOrcInputSplit(orcSplit, i)); + i++; + } + } + return list.toArray(new InceptorOrcInputSplit[i]); + } + + return null; + } + + @Override + public InputFormat createInputFormat() { + return new OrcInputFormat(); + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + try { + fs = + FileSystemUtil.getFileSystem( + inceptorFileConf.getHadoopConfig(), + inceptorFileConf.getDefaultFs(), + getRuntimeContext().getDistributedCache()); + } catch (Exception e) { + LOG.error( + "Get FileSystem error on openInputFormat() method, hadoopConfig = {}, Exception = {}", + inceptorFileConf.getHadoopConfig().toString(), + ExceptionUtil.getErrorMessage(e)); + } + } + + @Override + protected void openInternal(InputSplit inputSplit) throws IOException { + InceptorOrcInputSplit inceptorOrcInputSplit = (InceptorOrcInputSplit) inputSplit; + OrcSplit orcSplit = inceptorOrcInputSplit.getOrcSplit(); + + try { + if (!isInit.get()) { + init(orcSplit.getPath()); + isInit.set(true); + } + } catch (Exception e) { + throw new IOException("init [inspector] error", e); + } + ugi.doAs( + new PrivilegedAction() { + @Override + public Object run() { + if (inceptorFileConf.isTransaction()) { + try { + openAcidRecordReader(inputSplit); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + try { + openOrcReader(inputSplit); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + return null; + } + }); + } + + private void init(Path path) throws Exception { + OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(jobConf); + String typeStruct = + ugi.doAs( + new PrivilegedAction() { + @Override + public String run() { + try { + org.apache.hadoop.hive.ql.io.orc.Reader reader = + OrcFile.createReader(path, readerOptions); + return reader.getObjectInspector().getTypeName(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + if (StringUtils.isEmpty(typeStruct)) { + throw new RuntimeException("can't retrieve type struct from " + path); + } + LOG.info("original typeStruct {}", typeStruct); + + int startIndex = typeStruct.indexOf("<") + 1; + int endIndex = typeStruct.lastIndexOf(">"); + typeStruct = typeStruct.substring(startIndex, endIndex); + + List cols = parseColumnAndType(typeStruct); + + fullColNames = new String[cols.size()]; + String[] fullColTypes = new String[cols.size()]; + + for (int i = 0; i < cols.size(); ++i) { + int index = cols.get(i).indexOf(":"); + if (index > -1) { + fullColNames[i] = cols.get(i).substring(0, index); + fullColTypes[i] = cols.get(i).substring(index + 1); + } else { + LOG.warn("typeStruct {} is not valid", typeStruct); + } + } + + Properties p = new Properties(); + p.setProperty("columns", StringUtils.join(fullColNames, ",")); + p.setProperty("columns.types", StringUtils.join(fullColTypes, ":")); + + OrcSerde orcSerde = new OrcSerde(); + orcSerde.initialize(jobConf, p); + + this.inspector = (StructObjectInspector) orcSerde.getObjectInspector(); + } + + @Override + protected RowData nextRecordInternal(RowData rowData) throws ReadRecordException { + List fieldConfList = inceptorFileConf.getColumn(); + GenericRowData genericRowData; + if (fieldConfList.size() == 1 + && ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName())) { + genericRowData = new GenericRowData(fullColNames.length); + for (int i = 0; i < fullColNames.length; i++) { + Object col = inspector.getStructFieldData(value, fields.get(i)); + genericRowData.setField(i, InceptorUtil.getWritableValue(col)); + } + } else { + genericRowData = new GenericRowData(fieldConfList.size()); + for (int i = 0; i < fieldConfList.size(); i++) { + FieldConf metaColumn = fieldConfList.get(i); + Object val = null; + + if (metaColumn.getValue() != null) { + val = metaColumn.getValue(); + } else if (metaColumn.getIndex() != null + && metaColumn.getIndex() != -1 + && metaColumn.getIndex() < fullColNames.length) { + val = inspector.getStructFieldData(value, fields.get(metaColumn.getIndex())); + if (val == null && metaColumn.getValue() != null) { + val = metaColumn.getValue(); + } else { + val = InceptorUtil.getWritableValue(val); + } + } + + genericRowData.setField(i, val); + } + } + + try { + return rowConverter.toInternal(genericRowData); + } catch (Exception e) { + throw new ReadRecordException("", e, 0, rowData); + } + } + + /** + * parse column and type from orc type struct string + * + * @param typeStruct + * @return + */ + private List parseColumnAndType(String typeStruct) { + List columnList = new ArrayList<>(); + List splitList = Arrays.asList(typeStruct.split(ConstantValue.COMMA_SYMBOL)); + Iterator it = splitList.iterator(); + while (it.hasNext()) { + StringBuilder current = new StringBuilder(it.next()); + String currentStr = current.toString(); + boolean left = currentStr.contains(ConstantValue.LEFT_PARENTHESIS_SYMBOL); + boolean right = currentStr.contains(ConstantValue.RIGHT_PARENTHESIS_SYMBOL); + if (!left && !right) { + columnList.add(currentStr); + continue; + } + + if (left && right) { + columnList.add(currentStr); + continue; + } + + if (left && !right) { + while (it.hasNext()) { + String next = it.next(); + current.append(ConstantValue.COMMA_SYMBOL).append(next); + if (next.contains(ConstantValue.RIGHT_PARENTHESIS_SYMBOL)) { + break; + } + } + columnList.add(current.toString()); + } + } + return columnList; + } + + /** + * hive 事务表创建 RecordReader + * + * @param inputSplit + */ + private void openAcidRecordReader(InputSplit inputSplit) { + numReadCounter = getRuntimeContext().getLongCounter("numRead"); + InceptorOrcInputSplit hdfsOrcInputSplit = (InceptorOrcInputSplit) 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(jobConf); + readerOptions.filesystem(fs); + + org.apache.hadoop.hive.ql.io.orc.Reader reader = null; + 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); + } + LOG.info("original typeStruct {}", typeStruct); + + int startIndex = typeStruct.indexOf("<") + 1; + int endIndex = typeStruct.lastIndexOf(">"); + typeStruct = typeStruct.substring(startIndex, endIndex); + startIndex = typeStruct.indexOf("<") + 1; + endIndex = typeStruct.lastIndexOf(">"); + // typeStruct is + // "operation:int,originalTransaction:bigint,bucket:int,rowId:bigint,currentTransaction:bigint,row:struct<>" + if (startIndex != endIndex) { + LOG.info("before typeStruct {} ", typeStruct); + typeStruct = typeStruct.substring(startIndex, endIndex); + LOG.info("after typeStruct {} ", typeStruct); + } else { + LOG.warn("typeStruct {} ", typeStruct); + } + + // 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) { + int index = cols.get(i).indexOf(":"); + if (index > -1) { + fullColNames[i] = cols.get(i).substring(0, index); + fullColTypes[i] = cols.get(i).substring(index + 1); + } else { + LOG.warn("typeStruct {} is not valid", typeStruct); + } + } + + 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(jobConf, p); + + try { + this.inspector = (StructObjectInspector) orcSerde.getObjectInspector(); + } catch (SerDeException e) { + throw new RuntimeException("hive transaction table inspector create failed."); + } + // 读 hive 事务表需要设置的属性 + // jobConf.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, names); + jobConf.set("schema.evolution.columns", names); + // int:bigint:string:float:double:struct + // jobConf.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, types); + jobConf.set("schema.evolution.columns.types", types); + // AcidUtils.setAcidOperationalProperties(jobConf, true); + try { + recordReader = inputFormat.getRecordReader(orcSplit, jobConf, Reporter.NULL); + } catch (IOException e) { + throw new RuntimeException("hive transaction table record reader creation failed."); + } + key = recordReader.createKey(); + value = recordReader.createValue(); + fields = inspector.getAllStructFieldRefs(); + } + + private void openOrcReader(InputSplit inputSplit) throws IOException { + numReadCounter = getRuntimeContext().getLongCounter("numRead"); + InceptorOrcInputSplit hdfsOrcInputSplit = (InceptorOrcInputSplit) inputSplit; + OrcSplit orcSplit = hdfsOrcInputSplit.getOrcSplit(); + findCurrentPartition(orcSplit.getPath()); + recordReader = inputFormat.getRecordReader(orcSplit, jobConf, Reporter.NULL); + key = recordReader.createKey(); + value = recordReader.createValue(); + fields = inspector.getAllStructFieldRefs(); + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorParquetInputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorParquetInputFormat.java new file mode 100644 index 0000000000..d28d867c48 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorParquetInputFormat.java @@ -0,0 +1,426 @@ +/* + * 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.inceptor.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.inputSplit.InceptorParquetSplit; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.enums.ColumnType; +import com.dtstack.chunjun.throwable.ReadRecordException; +import com.dtstack.chunjun.util.ExceptionUtil; +import com.dtstack.chunjun.util.FileSystemUtil; +import com.dtstack.chunjun.util.PluginUtil; + +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.collect.Lists; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +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.concurrent.TimeUnit; + +public class InceptorParquetInputFormat extends BaseInceptorFileInputFormat { + + 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 currentSplitFilePaths; + private transient List fullColNames; + private transient List fullColTypes; + + private transient int currentFileIndex = 0; + + @Override + public InceptorParquetSplit[] createInceptorSplit(int minNumSplits) throws IOException { + super.initHadoopJobConf(); + String path; + if (StringUtils.isNotBlank(inceptorFileConf.getFileName())) { + // 兼容平台逻辑 + path = + inceptorFileConf.getPath() + + ConstantValue.SINGLE_SLASH_SYMBOL + + inceptorFileConf.getFileName(); + } else { + path = inceptorFileConf.getPath(); + } + + try { + FileSystem fs = + FileSystemUtil.getFileSystem( + inceptorFileConf.getHadoopConfig(), + inceptorFileConf.getDefaultFs(), + PluginUtil.createDistributedCacheFromContextClassLoader()); + HdfsPathFilter pathFilter = new HdfsPathFilter(inceptorFileConf.getFilterRegex()); + List allPartitionPath = getAllPartitionPath(path, fs, pathFilter); + + if (allPartitionPath.size() > 0) { + InceptorParquetSplit[] splits = new InceptorParquetSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new InceptorParquetSplit(i, new ArrayList<>()); + } + + Iterator it = allPartitionPath.iterator(); + while (it.hasNext()) { + for (InceptorParquetSplit split : splits) { + if (it.hasNext()) { + split.getPaths().add(it.next()); + } + } + } + + return splits; + } + } catch (Exception e) { + LOG.error( + "Get FileSystem error on openInputFormat() method, hadoopConfig = {}, Exception = {}", + inceptorFileConf.getHadoopConfig().toString(), + ExceptionUtil.getErrorMessage(e)); + } + + return null; + } + + @Override + public InputFormat createInputFormat() { + return null; + } + + @Override + protected void openInternal(InputSplit inputSplit) throws IOException { + currentSplitFilePaths = ((InceptorParquetSplit) inputSplit).getPaths(); + } + + @Override + public boolean reachedEnd() throws IOException { + return !nextLine(); + } + + @Override + protected RowData nextRecordInternal(RowData rowData) throws ReadRecordException { + List fieldConfList = inceptorFileConf.getColumn(); + GenericRowData genericRowData; + if (fieldConfList.size() == 1 + && ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName())) { + genericRowData = new GenericRowData(inceptorFileConf.getFullColumnName().size()); + for (int i = 0; i < inceptorFileConf.getFullColumnName().size(); i++) { + Object val = getData(currentLine, fullColTypes.get(i), i); + genericRowData.setField(i, val); + } + } else { + genericRowData = new GenericRowData(fieldConfList.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); + } + } + + private static List getAllPartitionPath( + String tableLocation, FileSystem fs, PathFilter pathFilter) throws IOException { + List pathList = Lists.newArrayList(); + Path inputPath = new Path(tableLocation); + + if (fs.isFile(inputPath)) { + pathList.add(tableLocation); + return pathList; + } + + FileStatus[] fsStatus = fs.listStatus(inputPath, pathFilter); + for (FileStatus status : fsStatus) { + pathList.addAll(getAllPartitionPath(status.getPath().toString(), fs, pathFilter)); + } + + return pathList; + } + + private boolean nextLine() throws IOException { + + getNextLine(); + if (currentLine != null) { + setMetaColumns(); + return true; + } + for (; currentFileIndex <= currentSplitFilePaths.size() - 1; ) { + if (openKerberos) { + ugi.doAs( + new PrivilegedAction() { + @Override + public Object run() { + try { + nextFile(); + return null; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + } else { + nextFile(); + } + getNextLine(); + if (currentLine != null) { + setMetaColumns(); + return true; + } + } + return false; + } + + @Override + public void closeInternal() throws IOException { + if (currentFileReader != null) { + currentFileReader.close(); + currentFileReader = null; + } + + currentLine = null; + currentFileIndex = 0; + } + + private 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 : inceptorFileConf.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; + } + + private void nextFile() throws IOException { + Path path = new Path(currentSplitFilePaths.get(currentFileIndex)); + findCurrentPartition(path); + ParquetReader.Builder reader = + ParquetReader.builder(new GroupReadSupport(), path).withConf(jobConf); + currentFileReader = reader.build(); + currentFileIndex++; + } + + private void getNextLine() throws IOException { + if (currentFileReader != null) { + currentLine = currentFileReader.read(); + } + } + + 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; + } + + 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); + } + + /** + * @param timestampBinary + * @return + */ + 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; + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorPathFilter.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorPathFilter.java new file mode 100644 index 0000000000..e1c00c4955 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorPathFilter.java @@ -0,0 +1,68 @@ +/* + * 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.inceptor.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; + +public class InceptorPathFilter 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 InceptorPathFilter() {} + + public InceptorPathFilter(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-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorSourceFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorSourceFactory.java index 90cf23382c..101b4a40a9 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorSourceFactory.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorSourceFactory.java @@ -19,34 +19,60 @@ package com.dtstack.chunjun.connector.inceptor.source; import com.dtstack.chunjun.conf.SyncConf; -import com.dtstack.chunjun.connector.inceptor.conf.InceptorConf; -import com.dtstack.chunjun.connector.inceptor.dialect.InceptorDialect; -import com.dtstack.chunjun.connector.inceptor.util.InceptorDbUtil; -import com.dtstack.chunjun.connector.jdbc.conf.JdbcConf; -import com.dtstack.chunjun.connector.jdbc.source.JdbcInputFormatBuilder; -import com.dtstack.chunjun.connector.jdbc.source.JdbcSourceFactory; +import com.dtstack.chunjun.converter.RawTypeConverter; +import com.dtstack.chunjun.source.SourceFactory; +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/2/22 */ -public class InceptorSourceFactory extends JdbcSourceFactory { +import org.apache.commons.collections.MapUtils; + +import java.util.Collections; +import java.util.List; +import java.util.Map; - InceptorDialect inceptorDialect; +/** @author liuliu 2022/2/22 */ +public class InceptorSourceFactory extends SourceFactory { + private SourceFactory sourceFactory; public InceptorSourceFactory(SyncConf syncConf, StreamExecutionEnvironment env) { - super(syncConf, env, null); - this.inceptorDialect = InceptorDbUtil.getDialectWithDriverType(jdbcConf); - jdbcConf.setJdbcUrl(inceptorDialect.appendJdbcTransactionType(jdbcConf.getJdbcUrl())); - super.jdbcDialect = inceptorDialect; + super(syncConf, env); + boolean useJdbc = !syncConf.getReader().getParameter().containsKey("path"); + + boolean transaction = + MapUtils.getBoolean(syncConf.getReader().getParameter(), "isTransaction", false); + // 事务表直接jdbc读取 + if (useJdbc || transaction) { + refactorConf(syncConf); + this.sourceFactory = new InceptorJdbcSourceFactory(syncConf, env); + } else { + this.sourceFactory = new InceptorFileSourceFactory(syncConf, env); + } } @Override - protected Class getConfClass() { - return InceptorConf.class; + public RawTypeConverter getRawTypeConverter() { + return sourceFactory.getRawTypeConverter(); } @Override - protected JdbcInputFormatBuilder getBuilder() { - return inceptorDialect.getInputFormatBuilder(); + public DataStream createSource() { + return sourceFactory.createSource(); + } + + private void refactorConf(SyncConf syncConf) { + if (syncConf.getReader().getParameter().containsKey("connection")) { + Object connection = syncConf.getReader().getParameter().get("connection"); + if (connection instanceof List) { + List> connections = (List>) connection; + connections.forEach( + i -> { + if (i.get("jdbcUrl") != null && i.get("jdbcUrl") instanceof String) { + i.put("jdbcUrl", Collections.singletonList(i.get("jdbcUrl"))); + } + }); + } + } } } diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorTextInputFormat.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorTextInputFormat.java new file mode 100644 index 0000000000..0018bd6c88 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorTextInputFormat.java @@ -0,0 +1,143 @@ +/* + * 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.inceptor.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.inputSplit.InceptorTextInputSplit; +import com.dtstack.chunjun.connector.inceptor.util.InceptorUtil; +import com.dtstack.chunjun.constants.ConstantValue; +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.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.security.PrivilegedAction; +import java.util.List; + +public class InceptorTextInputFormat extends BaseInceptorFileInputFormat { + + @Override + public InputSplit[] createInceptorSplit(int minNumSplits) throws IOException { + super.initHadoopJobConf(); + org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter( + jobConf, InceptorPathFilter.class); + + org.apache.hadoop.mapred.FileInputFormat.setInputPaths(jobConf, inceptorFileConf.getPath()); + TextInputFormat inputFormat = new TextInputFormat(); + + // 是否在MapReduce中递归遍历Input目录 + jobConf.set("mapreduce.input.fileinputformat.input.dir.recursive", "true"); + inputFormat.configure(jobConf); + org.apache.hadoop.mapred.InputSplit[] splits = inputFormat.getSplits(jobConf, minNumSplits); + + if (splits != null) { + InceptorTextInputSplit[] hdfsTextInputSplits = + new InceptorTextInputSplit[splits.length]; + for (int i = 0; i < splits.length; ++i) { + hdfsTextInputSplits[i] = new InceptorTextInputSplit(splits[i], i); + } + return hdfsTextInputSplits; + } + return null; + } + + @Override + public InputFormat createInputFormat() { + return new TextInputFormat(); + } + + @Override + protected void openInternal(InputSplit inputSplit) throws IOException { + ugi.doAs( + new PrivilegedAction() { + @Override + public Object run() { + try { + openHdfsTextReader(inputSplit); + } catch (Exception e) { + throw new RuntimeException(e); + } + + return null; + } + }); + } + + @Override + protected RowData nextRecordInternal(RowData rowData) throws ReadRecordException { + try { + String line = + new String( + ((Text) value).getBytes(), + 0, + ((Text) value).getLength(), + inceptorFileConf.getEncoding()); + String[] fields = + StringUtils.splitPreserveAllTokens(line, inceptorFileConf.getFieldDelimiter()); + + List fieldConfList = inceptorFileConf.getColumn(); + GenericRowData genericRowData; + if (fieldConfList.size() == 1 + && ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName())) { + genericRowData = new GenericRowData(fields.length); + for (int i = 0; i < fields.length; i++) { + genericRowData.setField(i, fields[i]); + } + } else { + genericRowData = new GenericRowData(fieldConfList.size()); + 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 (!InceptorUtil.NULL_VALUE.equals(strVal)) { + value = strVal; + } + } + genericRowData.setField(i, value); + } + } + return rowConverter.toInternal(genericRowData); + } catch (Exception e) { + throw new ReadRecordException("", e, 0, rowData); + } + } + + private void openHdfsTextReader(InputSplit inputSplit) throws IOException { + InceptorTextInputSplit hdfsTextInputSplit = (InceptorTextInputSplit) inputSplit; + org.apache.hadoop.mapred.InputSplit fileSplit = hdfsTextInputSplit.getTextSplit(); + findCurrentPartition(((FileSplit) fileSplit).getPath()); + super.recordReader = super.inputFormat.getRecordReader(fileSplit, jobConf, Reporter.NULL); + super.key = new LongWritable(); + super.value = new Text(); + } +} diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/util/InceptorDbUtil.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/util/InceptorDbUtil.java index dd43bb19dd..1dade110e3 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/util/InceptorDbUtil.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/util/InceptorDbUtil.java @@ -206,7 +206,9 @@ public static String getTableStorageHandler(JdbcConf jdbcConf) { String storageHandler = ""; try { - connection.createStatement().execute(String.format("use %s", schema)); + if (StringUtils.isNotBlank(schema)) { + connection.createStatement().execute(String.format("use %s", schema)); + } String sql = String.format(INCEPROE_SHOW_DESCRIBE_FORMAT, table); ResultSet resultSet = connection.createStatement().executeQuery(sql); while (resultSet.next()) { diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/util/InceptorUtil.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/util/InceptorUtil.java new file mode 100644 index 0000000000..622242ad46 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/util/InceptorUtil.java @@ -0,0 +1,484 @@ +/* + * 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.inceptor.util; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorOrcColumnConvent; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorOrcRowConverter; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorParquetColumnConverter; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorParquetRowConverter; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorTextColumnConvent; +import com.dtstack.chunjun.connector.inceptor.converter.InceptorTextRowConverter; +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.util.TableUtil; + +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +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.ByteWritable; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.HiveVarchar2Writable; +import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.BytesWritable; +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.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.security.PrivilegedAction; +import java.sql.Timestamp; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static com.dtstack.chunjun.security.KerberosUtil.KRB_STR; + +public class InceptorUtil { + public static final Logger LOG = LoggerFactory.getLogger(InceptorUtil.class); + public static final String NULL_VALUE = "\\N"; + + private static final double SCALE_TWO = 2.0; + private static final double SCALE_TEN = 10.0; + private static final int BIT_SIZE = 8; + + public static final int JULIAN_EPOCH_OFFSET_DAYS = 2440588; + public static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1L); + public static final long NANOS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1L); + public static final long NANOS_PER_SECOND = TimeUnit.SECONDS.toNanos(1L); + 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"; + private static final String KEY_HADOOP_SECURITY_AUTHORIZATION = "hadoop.security.authorization"; + private static final String KEY_HADOOP_SECURITY_AUTHENTICATION = + "hadoop.security.authentication"; + + /** + * createRowConverter + * + * @param useAbstractBaseColumn + * @param fileType + * @param fieldConfList + * @param converter + * @return + */ + public static AbstractRowConverter createRowConverter( + boolean useAbstractBaseColumn, + String fileType, + List fieldConfList, + RawTypeConverter converter) { + AbstractRowConverter rowConverter; + if (useAbstractBaseColumn) { + if ("ORC".equals(fileType.toUpperCase(Locale.ENGLISH))) { + rowConverter = new InceptorOrcColumnConvent(fieldConfList); + } else if ("PARQUET".equals(fileType.toUpperCase(Locale.ENGLISH))) { + rowConverter = new InceptorParquetColumnConverter(fieldConfList); + } else { + rowConverter = new InceptorTextColumnConvent(fieldConfList); + } + } else { + RowType rowType = TableUtil.createRowType(fieldConfList, converter); + switch (fileType.toUpperCase(Locale.ENGLISH)) { + case "ORC": + rowConverter = new InceptorOrcRowConverter(rowType); + break; + case "PARQUET": + rowConverter = new InceptorParquetRowConverter(rowType); + break; + default: + rowConverter = new InceptorTextRowConverter(rowType); + break; + } + } + return rowConverter; + } + + public static Object getWritableValue(Object writable) { + if (writable == null) { + return null; + } + Class clz = writable.getClass(); + Object ret = null; + if (clz == IntWritable.class) { + ret = ((IntWritable) writable).get(); + } else if (clz == Text.class) { + ret = ((Text) 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) { + // tdh 日期类型为hivedate,要转化成Date + ret = new java.util.Date(((DateWritable) writable).get().getTime()); + } else if (writable instanceof DoubleWritable) { + ret = ((DoubleWritable) writable).get(); + } else if (writable instanceof TimestampWritable) { + ret = ((TimestampWritable) writable).getTimestamp(); + } else if (writable instanceof DateWritable) { + ret = new java.util.Date(((DateWritable) writable).get().getTime()); + } 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 if (writable instanceof ByteWritable) { + ByteWritable byteWritable = (ByteWritable) ret; + ret = String.valueOf(byteWritable.get()); + } else { + ret = writable.toString(); + } + return ret; + } + + /** + * 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 (org.apache.commons.lang3.StringUtils.isNotBlank(customizeMessage)) { + str.append(customizeMessage); + } + if (org.apache.commons.lang3.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 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 columnTypeToObjectInspetor(ColumnType columnType) { + ObjectInspector objectInspector = null; + switch (columnType) { + case TINYINT: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + ByteWritable.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case SMALLINT: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + ShortWritable.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( + DoubleWritable.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DECIMAL: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + HiveDecimalWritable.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case TIMESTAMP: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + Timestamp.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DATE: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + DateWritable.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case STRING: + case CHAR: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + String.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case VARCHAR2: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + HiveVarchar2Writable.class, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case VARCHAR: + objectInspector = + ObjectInspectorFactory.getReflectionObjectInspector( + HiveVarcharWritable.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 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 Binary timestampToInt96(TimestampData timestampData) { + int julianDay; + long nanosOfDay; + + // Use UTC timezone or local timezone to the conversion between epoch time and + // LocalDateTime. + // Hive 0.x/1.x/2.x use local timezone. But Hive 3.x use UTC timezone. + Timestamp timestamp = timestampData.toTimestamp(); + long mills = timestamp.getTime(); + julianDay = (int) ((mills / MILLIS_IN_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + nanosOfDay = ((mills % MILLIS_IN_DAY) / 1000) * NANOS_PER_SECOND + timestamp.getNanos(); + + ByteBuffer buf = ByteBuffer.allocate(12); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.putLong(nanosOfDay); + buf.putInt(julianDay); + buf.flip(); + return Binary.fromConstantByteBuffer(buf); + } + + 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)); + } + + 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 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); + } + + 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 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; + } + + 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; + } + + public static void setHadoopUserName(Configuration conf) { + String hadoopUserName = conf.get(KEY_HADOOP_USER_NAME); + if (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); + } catch (Exception e) { + LOG.warn("Set hadoop user name error:", e); + } + } + + private static boolean isHaMode(Map confMap) { + return StringUtils.isNotEmpty(MapUtils.getString(confMap, KEY_DFS_NAME_SERVICES)); + } +} diff --git a/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/util/SqlUtil.java b/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/util/SqlUtil.java index 7315fcdf68..ae9ce43c95 100644 --- a/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/util/SqlUtil.java +++ b/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/util/SqlUtil.java @@ -253,10 +253,15 @@ public static String buildSplitFilterSql( JdbcDialect jdbcDialect, JdbcInputSplit jdbcInputSplit, String splitColumn) { + String sql; if ("range".equalsIgnoreCase(splitStrategy)) { - return jdbcDialect.getSplitRangeFilter(jdbcInputSplit, splitColumn); + sql = jdbcDialect.getSplitRangeFilter(jdbcInputSplit, splitColumn); } else { - return jdbcDialect.getSplitModFilter(jdbcInputSplit, splitColumn); + sql = jdbcDialect.getSplitModFilter(jdbcInputSplit, splitColumn); } + if (jdbcInputSplit.getSplitNumber() == 0) { + sql += " OR " + jdbcDialect.quoteIdentifier(splitColumn) + " IS NULL"; + } + return sql; } } diff --git a/chunjun-connectors/pom.xml b/chunjun-connectors/pom.xml index bf2ecbddbf..952f3ae728 100644 --- a/chunjun-connectors/pom.xml +++ b/chunjun-connectors/pom.xml @@ -48,9 +48,9 @@ chunjun-connector-file chunjun-connector-filesystem chunjun-connector-ftp - chunjun-connector-hdfs - chunjun-connector-hive - chunjun-connector-inceptor + + + chunjun-connector-hbase-base diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/BaseMetric.java b/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/BaseMetric.java index 6baecf8bb7..b4fa29d7e6 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/BaseMetric.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/BaseMetric.java @@ -39,7 +39,7 @@ public class BaseMetric { protected final Logger LOG = LoggerFactory.getLogger(getClass()); - private final Long delayPeriodMill = 10000L; + private final Long delayPeriodMill = 20000L; private final MetricGroup chunjunMetricGroup; diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/util/FileSystemUtil.java b/chunjun-core/src/main/java/com/dtstack/chunjun/util/FileSystemUtil.java index dc37233922..03d74658ec 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/util/FileSystemUtil.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/util/FileSystemUtil.java @@ -128,6 +128,20 @@ public static UserGroupInformation getUGI( getConfiguration(hadoopConfig, defaultFs), principal, keytabFileName); } + public static String getHadoopUser(Map hadoopConfig) throws IOException { + Object hadoopUserName = hadoopConfig.get("hadoop.user.name"); + if (hadoopUserName != null && StringUtils.isNotBlank(hadoopUserName.toString())) { + return hadoopUserName.toString(); + } + + hadoopUserName = hadoopConfig.get("HADOOP_USER_NAME"); + if (hadoopUserName != null && StringUtils.isNotBlank(hadoopUserName.toString())) { + return hadoopUserName.toString(); + } + + return UserGroupInformation.getCurrentUser().getUserName(); + } + public static Configuration getConfiguration(Map confMap, String defaultFs) { confMap = fillConfig(confMap, defaultFs); diff --git a/pom.xml b/pom.xml index bf9a56e769..00f3acf95a 100644 --- a/pom.xml +++ b/pom.xml @@ -423,4 +423,42 @@ https://github.com/dtstack/maven-repository/raw/master/ + + + + + local-test + + chunjun-local-test + + + + + + default + + true + + + 2.7.5 + + + + chunjun-connectors/chunjun-connector-hdfs + chunjun-connectors/chunjun-connector-hive + + + + + + + tdh + + + chunjun-connectors/chunjun-connector-inceptor + + + + +