Skip to content
Open
Show file tree
Hide file tree
Changes from 2 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 @@ -32,6 +32,8 @@
import com.alibaba.fluss.utils.ExceptionUtils;
import com.alibaba.fluss.utils.IOUtils;

import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.Schema;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.catalog.CatalogBaseTable;
import org.apache.flink.table.catalog.CatalogDatabase;
Expand Down Expand Up @@ -63,6 +65,7 @@

import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
Expand All @@ -78,6 +81,8 @@ public class FlinkCatalog implements Catalog {

public static final String LAKE_TABLE_SPLITTER = "$lake";

public static final String CHANGELOG_TABLE_SPLITTER = "$changelog";

protected final ClassLoader classLoader;

protected final String catalogName;
Expand Down Expand Up @@ -263,6 +268,48 @@ public CatalogBaseTable getTable(ObjectPath objectPath)
tableName.split("\\" + LAKE_TABLE_SPLITTER)[0])));
}
return getLakeTable(objectPath.getDatabaseName(), tableName);
} else if ((tableName.contains(CHANGELOG_TABLE_SPLITTER))) {
String baseTableName = tableName.split("\\" + CHANGELOG_TABLE_SPLITTER)[0];
TablePath baseTablePath = TablePath.of(objectPath.getDatabaseName(), baseTableName);
tableInfo = admin.getTableInfo(baseTablePath).get();
if (!tableInfo.hasPrimaryKey()) {
throw new UnsupportedOperationException(
String.format(
"\"Table %s has no primary key, only primary key tables support changelog.",
baseTableName));
}
CatalogTable originalTable = FlinkConversions.toFlinkTable(tableInfo);
Schema originalSchema = originalTable.getUnresolvedSchema();
List<Schema.UnresolvedColumn> newColumns =
new ArrayList<>(originalSchema.getColumns());
newColumns.add(
new Schema.UnresolvedPhysicalColumn("_change_type", DataTypes.STRING()));
newColumns.add(
new Schema.UnresolvedPhysicalColumn("_log_offset", DataTypes.BIGINT()));
newColumns.add(
new Schema.UnresolvedPhysicalColumn(
"_commit_timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()));
Schema changeLogSchema =
Schema.newBuilder()
.fromColumns(newColumns)
.primaryKey(
originalSchema
.getPrimaryKey()
.orElse(null)
.getColumnNames())
.build();

Map<String, String> options = new HashMap<>(originalTable.getOptions());
options.put(BOOTSTRAP_SERVERS.key(), bootstrapServers);
options.put("changelog", "true");

return CatalogTable.newBuilder()
.schema(changeLogSchema)
.comment(originalTable.getComment())
.options(options)
.partitionKeys(originalTable.getPartitionKeys())
.build();

} else {
tableInfo = admin.getTableInfo(tablePath).get();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import java.util.HashSet;
import java.util.Set;

import static com.alibaba.fluss.connector.flink.catalog.FlinkCatalog.CHANGELOG_TABLE_SPLITTER;
import static com.alibaba.fluss.connector.flink.catalog.FlinkCatalog.LAKE_TABLE_SPLITTER;
import static com.alibaba.fluss.connector.flink.utils.FlinkConversions.toFlinkOption;

Expand All @@ -69,6 +70,10 @@ public DynamicTableSource createDynamicTableSource(Context context) {
lakeTableFactory = mayInitLakeTableFactory();
return lakeTableFactory.createDynamicTableSource(context, tableName);
}
boolean isChangelog = false;
if (tableName.contains(CHANGELOG_TABLE_SPLITTER)) {
isChangelog = true;
}

FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
helper.validateExcept("table.", "client.");
Expand Down Expand Up @@ -129,7 +134,8 @@ public DynamicTableSource createDynamicTableSource(Context context) {
cache,
partitionDiscoveryIntervalMs,
tableOptions.get(toFlinkOption(ConfigOptions.TABLE_DATALAKE_ENABLED)),
tableOptions.get(toFlinkOption(ConfigOptions.TABLE_MERGE_ENGINE)));
tableOptions.get(toFlinkOption(ConfigOptions.TABLE_MERGE_ENGINE)),
isChangelog);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ public class FlinkSource implements Source<RowData, SourceSplitBase, SourceEnume
private final OffsetsInitializer offsetsInitializer;
private final long scanPartitionDiscoveryIntervalMs;
private final boolean streaming;
private boolean isChangelog;

public FlinkSource(
Configuration flussConf,
Expand All @@ -65,7 +66,8 @@ public FlinkSource(
@Nullable int[] projectedFields,
OffsetsInitializer offsetsInitializer,
long scanPartitionDiscoveryIntervalMs,
boolean streaming) {
boolean streaming,
boolean isChangelog) {
this.flussConf = flussConf;
this.tablePath = tablePath;
this.hasPrimaryKey = hasPrimaryKey;
Expand All @@ -75,6 +77,7 @@ public FlinkSource(
this.offsetsInitializer = offsetsInitializer;
this.scanPartitionDiscoveryIntervalMs = scanPartitionDiscoveryIntervalMs;
this.streaming = streaming;
this.isChangelog = isChangelog;
}

@Override
Expand Down Expand Up @@ -136,6 +139,7 @@ public SourceReader<RowData, SourceSplitBase> createReader(SourceReaderContext c
sourceOutputType,
context,
projectedFields,
flinkSourceReaderMetrics);
flinkSourceReaderMetrics,
isChangelog);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ public class FlinkTableSource
// will be empty if no partition key
private final int[] partitionKeyIndexes;
private final boolean streaming;
private final boolean isChangelog;
private final FlinkConnectorOptionsUtils.StartupOptions startupOptions;

// options for lookup source
Expand Down Expand Up @@ -137,7 +138,8 @@ public FlinkTableSource(
@Nullable LookupCache cache,
long scanPartitionDiscoveryIntervalMs,
boolean isDataLakeEnabled,
@Nullable MergeEngineType mergeEngineType) {
@Nullable MergeEngineType mergeEngineType,
boolean isChangelog) {
this.tablePath = tablePath;
this.flussConfig = flussConfig;
this.tableOutputType = tableOutputType;
Expand All @@ -155,6 +157,7 @@ public FlinkTableSource(
this.scanPartitionDiscoveryIntervalMs = scanPartitionDiscoveryIntervalMs;
this.isDataLakeEnabled = isDataLakeEnabled;
this.mergeEngineType = mergeEngineType;
this.isChangelog = isChangelog;
}

@Override
Expand Down Expand Up @@ -261,7 +264,8 @@ public boolean isBounded() {
projectedFields,
offsetsInitializer,
scanPartitionDiscoveryIntervalMs,
streaming);
streaming,
isChangelog);

if (!streaming) {
// return a bounded source provide to make planner happy,
Expand Down Expand Up @@ -350,7 +354,8 @@ public DynamicTableSource copy() {
cache,
scanPartitionDiscoveryIntervalMs,
isDataLakeEnabled,
mergeEngineType);
mergeEngineType,
isChangelog);
source.producedDataType = producedDataType;
source.projectedFields = projectedFields;
source.singleRowFilter = singleRowFilter;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.alibaba.fluss.connector.flink.source.reader.RecordAndPos;
import com.alibaba.fluss.connector.flink.source.split.HybridSnapshotLogSplitState;
import com.alibaba.fluss.connector.flink.source.split.SourceSplitState;
import com.alibaba.fluss.connector.flink.utils.ChangelogRowConverter;
import com.alibaba.fluss.connector.flink.utils.FlussRowToFlinkRowConverter;
import com.alibaba.fluss.types.RowType;

Expand All @@ -48,8 +49,12 @@ public class FlinkRecordEmitter implements RecordEmitter<RecordAndPos, RowData,

private LakeRecordRecordEmitter lakeRecordRecordEmitter;

public FlinkRecordEmitter(RowType rowType) {
this.converter = new FlussRowToFlinkRowConverter(rowType);
public FlinkRecordEmitter(RowType rowType, boolean isChangeLogMode) {
if (!isChangeLogMode) {
this.converter = new FlussRowToFlinkRowConverter(rowType);
} else {
this.converter = new ChangelogRowConverter(rowType);
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,15 +48,15 @@
public class FlinkSourceReader
extends SingleThreadMultiplexSourceReaderBase<
RecordAndPos, RowData, SourceSplitBase, SourceSplitState> {

public FlinkSourceReader(
FutureCompletingBlockingQueue<RecordsWithSplitIds<RecordAndPos>> elementsQueue,
Configuration flussConfig,
TablePath tablePath,
RowType sourceOutputType,
SourceReaderContext context,
@Nullable int[] projectedFields,
FlinkSourceReaderMetrics flinkSourceReaderMetrics) {
FlinkSourceReaderMetrics flinkSourceReaderMetrics,
boolean isChangelog) {
super(
elementsQueue,
new FlinkSourceFetcherManager(
Expand All @@ -69,7 +69,8 @@ public FlinkSourceReader(
projectedFields,
flinkSourceReaderMetrics),
(ignore) -> {}),
new FlinkRecordEmitter(sourceOutputType),
// InternalRow into Flink RowData with the additional metadata columns
new FlinkRecordEmitter(sourceOutputType, isChangelog),
context.getConfiguration(),
context);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* Copyright (c) 2024 Alibaba Group Holding Ltd.
*
* 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
*
* 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.alibaba.fluss.connector.flink.utils;

import com.alibaba.fluss.client.table.scanner.ScanRecord;
import com.alibaba.fluss.types.RowType;

import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;

public class ChangelogRowConverter extends FlussRowToFlinkRowConverter {
public ChangelogRowConverter(RowType rowType) {
super(rowType);
}

public RowData toFlinkRowData(ScanRecord scanRecord) {
RowData baseRowData = super.toFlinkRowData(scanRecord);
GenericRowData rowWithMetadata = new GenericRowData(baseRowData.getArity() + 3);
rowWithMetadata.setRowKind(baseRowData.getRowKind());

for (int i = 0; i < baseRowData.getArity(); i++) {
rowWithMetadata.setField(i, baseRowData.getRawValue(i));
}

int baseArity = baseRowData.getArity();
String changeType;
switch (scanRecord.getRowKind()) {
case INSERT:
changeType = "+I";
break;
case UPDATE_BEFORE:
changeType = "-U";
break;
case UPDATE_AFTER:
changeType = "+U";
break;
case DELETE:
changeType = "-D";
break;
default:
changeType = "+I";
break;
}
rowWithMetadata.setField(baseArity, changeType);
rowWithMetadata.setField(baseArity + 1, scanRecord.logOffset());
rowWithMetadata.setField(baseArity + 2, scanRecord.timestamp());

return rowWithMetadata;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,8 @@
import javax.annotation.Nullable;

import java.io.UncheckedIOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.function.Supplier;
Expand Down Expand Up @@ -143,6 +145,17 @@ public CompletableFuture<CreateTableResponse> createTable(CreateTableRequest req
TableDescriptor tableDescriptor;
try {
tableDescriptor = TableDescriptor.fromJsonBytes(request.getTableJson());
// Validate reserved column names
List<String> reservedColumns =
Arrays.asList("_change_type", "_log_offset", "_commit_timestamp");
for (String columnName : tableDescriptor.getSchema().getColumnNames()) {
if (reservedColumns.contains(columnName)) {
throw new InvalidTableException(
String.format(
"Column name '%s' is reserved for system use.", columnName));
}
}

} catch (Exception e) {
if (e instanceof UncheckedIOException) {
throw new InvalidTableException(
Expand Down
Loading