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