Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[hotfix-#1256][kudu]Fix SQLMode datatype mapping #1259

Merged
merged 1 commit into from
Sep 16, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.dtstack.chunjun.connector.kudu.converter.KuduColumnConverter;
import com.dtstack.chunjun.connector.kudu.converter.KuduRawTypeConverter;
import com.dtstack.chunjun.connector.kudu.util.KuduUtil;
import com.dtstack.chunjun.converter.AbstractRowConverter;
import com.dtstack.chunjun.source.format.BaseRichInputFormat;
import com.dtstack.chunjun.throwable.ReadRecordException;
import com.dtstack.chunjun.util.TableUtil;
Expand Down Expand Up @@ -86,15 +87,13 @@ protected void openInternal(InputSplit inputSplit) throws IOException {
KuduInputSplit kuduTableSplit = (KuduInputSplit) inputSplit;
scanner = KuduScanToken.deserializeIntoScanner(kuduTableSplit.getToken(), client);

List<String> columnNames = new ArrayList<>();
List<FieldConf> fieldConfList = sourceConf.getColumn();
fieldConfList.forEach(field -> columnNames.add(field.getName()));
RowType rowType = TableUtil.createRowType(fieldConfList, KuduRawTypeConverter::apply);

setRowConverter(
rowConverter == null
? new KuduColumnConverter(rowType, columnNames)
: rowConverter);
if (rowConverter == null) {
List<String> columnNames = new ArrayList<>();
List<FieldConf> fieldConfList = sourceConf.getColumn();
fieldConfList.forEach(field -> columnNames.add(field.getName()));
RowType rowType = TableUtil.createRowType(fieldConfList, KuduRawTypeConverter::apply);
setRowConverter(new KuduColumnConverter(rowType, columnNames));
}
}

@Override
Expand Down Expand Up @@ -157,4 +156,8 @@ public void setSourceConf(KuduSourceConf sourceConf) {
public KuduSourceConf getSourceConf() {
return sourceConf;
}

public AbstractRowConverter getRowConverter() {
return rowConverter;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,9 @@ protected void checkFormat() {
StringBuilder sb = new StringBuilder(256);

if (columns == null || columns.size() == 0) {
sb.append("Columns can not be empty.\n");
if (format.getRowConverter() == null) {
sb.append("At least one of the Column and rowConverter is not empty.\n");
}
}

if (sourceConf.getBatchSizeBytes() > ConstantValue.STORE_SIZE_G) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,24 +18,18 @@

package com.dtstack.chunjun.connector.kudu.table;

import com.dtstack.chunjun.conf.FieldConf;
import com.dtstack.chunjun.connector.kudu.conf.KuduSinkConf;
import com.dtstack.chunjun.connector.kudu.converter.KuduRawTypeConverter;
import com.dtstack.chunjun.connector.kudu.converter.KuduRowConverter;
import com.dtstack.chunjun.connector.kudu.sink.KuduOutputFormatBuilder;
import com.dtstack.chunjun.sink.DtOutputFormatSinkFunction;
import com.dtstack.chunjun.util.TableUtil;

import org.apache.flink.table.api.TableSchema;
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.AtomicDataType;
import org.apache.flink.table.types.logical.NullType;
import org.apache.flink.table.types.logical.RowType;

import java.util.ArrayList;
import java.util.List;
import java.util.Arrays;

/**
* @author tiezhu
Expand Down Expand Up @@ -64,33 +58,11 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
KuduOutputFormatBuilder builder = new KuduOutputFormatBuilder();

String[] fieldNames = tableSchema.getFieldNames();
List<FieldConf> columnList = new ArrayList<>(fieldNames.length);
List<String> columnNameList = new ArrayList<>();

for (int index = 0; index < fieldNames.length; index++) {
String name = fieldNames[index];
columnNameList.add(name);

FieldConf field = new FieldConf();
field.setName(name);
field.setType(
tableSchema
.getFieldDataType(name)
.orElse(new AtomicDataType(new NullType()))
.getLogicalType()
.getTypeRoot()
.name());
field.setIndex(index);
columnList.add(field);
}
sinkConf.setColumn(columnList);

final RowType rowType =
TableUtil.createRowType(sinkConf.getColumn(), KuduRawTypeConverter::apply);

builder.setSinkConf(sinkConf);
builder.setRowConverter(new KuduRowConverter(rowType, columnNameList));
builder.setRowConverter(
new KuduRowConverter(
(RowType) tableSchema.toRowDataType().getLogicalType(),
Arrays.asList(tableSchema.getFieldNames())));

return SinkFunctionProvider.of(
new DtOutputFormatSinkFunction(builder.finish()), sinkConf.getParallelism());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,8 @@

package com.dtstack.chunjun.connector.kudu.table;

import com.dtstack.chunjun.conf.FieldConf;
import com.dtstack.chunjun.connector.kudu.conf.KuduLookupConf;
import com.dtstack.chunjun.connector.kudu.conf.KuduSourceConf;
import com.dtstack.chunjun.connector.kudu.converter.KuduRawTypeConverter;
import com.dtstack.chunjun.connector.kudu.converter.KuduRowConverter;
import com.dtstack.chunjun.connector.kudu.source.KuduInputFormatBuilder;
import com.dtstack.chunjun.connector.kudu.table.lookup.KuduAllTableFunction;
Expand All @@ -31,7 +29,6 @@
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.util.TableUtil;

import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connector.ChangelogMode;
Expand All @@ -41,16 +38,11 @@
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.AtomicDataType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.NullType;
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.Arrays;
import java.util.List;

/**
* @author tiezhu
Expand Down Expand Up @@ -78,37 +70,12 @@ public KuduDynamicTableSource(
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
KuduInputFormatBuilder builder = new KuduInputFormatBuilder();

LogicalType logicalType = tableSchema.toRowDataType().getLogicalType();
InternalTypeInfo<RowData> typeInfo = InternalTypeInfo.of(logicalType);

String[] fieldNames = tableSchema.getFieldNames();
List<FieldConf> columnList = new ArrayList<>(fieldNames.length);
List<String> columnNameList = new ArrayList<>();

for (int index = 0; index < fieldNames.length; index++) {
String name = fieldNames[index];
columnNameList.add(name);

FieldConf field = new FieldConf();
field.setName(name);
field.setType(
tableSchema
.getFieldDataType(name)
.orElse(new AtomicDataType(new NullType()))
.getLogicalType()
.getTypeRoot()
.name());
field.setIndex(index);
columnList.add(field);
}

sourceConf.setColumn(columnList);

RowType rowType =
TableUtil.createRowType(sourceConf.getColumn(), KuduRawTypeConverter::apply);
RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
InternalTypeInfo<RowData> typeInfo = InternalTypeInfo.of(rowType);

builder.setKuduSourceConf(sourceConf);
builder.setRowConverter(new KuduRowConverter(rowType, columnNameList));
builder.setRowConverter(
new KuduRowConverter(rowType, Arrays.asList(tableSchema.getFieldNames())));

return ParallelSourceFunctionProvider.of(
new DtInputFormatSourceFunction<>(builder.finish(), typeInfo),
Expand Down