Skip to content

Commit

Permalink
fix checkstyle
Browse files Browse the repository at this point in the history
  • Loading branch information
wangshikai committed Jul 15, 2024
1 parent 557f4a6 commit 0dca9c7
Show file tree
Hide file tree
Showing 16 changed files with 273 additions and 173 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig;
import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory;
import org.apache.flink.cdc.connectors.mysql.source.reader.MySqlPipelineRecordEmitter;
import org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReaderContext;
import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode;

/** A {@link DataSource} for mysql cdc connector. */
Expand All @@ -54,7 +53,10 @@ public EventSourceProvider getEventSourceProvider() {
deserializer,
(sourceReaderContext, sourceReaderMetrics, sourceConfig) ->
new MySqlPipelineRecordEmitter(
sourceReaderContext, deserializer, sourceReaderMetrics, sourceConfig));
sourceReaderContext,
deserializer,
sourceReaderMetrics,
sourceConfig));

return FlinkSourceProvider.of(source);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,16 +64,12 @@ public final class RowDataDebeziumDeserializeSchema
implements DebeziumDeserializationSchema<RowData> {
private static final long serialVersionUID = 2L;

/**
* Custom validator to validate the row value.
*/
/** Custom validator to validate the row value. */
public interface ValueValidator extends Serializable {
void validate(RowData rowData, RowKind rowKind) throws Exception;
}

/**
* TypeInformation of the produced {@link RowData}. *
*/
/** TypeInformation of the produced {@link RowData}. * */
private final TypeInformation<RowData> resultTypeInfo;

/**
Expand All @@ -82,29 +78,21 @@ public interface ValueValidator extends Serializable {
*/
private final DeserializationRuntimeConverter physicalConverter;

/**
* Whether the deserializer needs to handle metadata columns.
*/
/** Whether the deserializer needs to handle metadata columns. */
private final boolean hasMetadata;

/**
* A wrapped output collector which is used to append metadata columns after physical columns.
*/
private final AppendMetadataCollector appendMetadataCollector;

/**
* Validator to validate the row value.
*/
/** Validator to validate the row value. */
private final ValueValidator validator;

/**
* Changelog Mode to use for encoding changes in Flink internal data structure.
*/
/** Changelog Mode to use for encoding changes in Flink internal data structure. */
private final DebeziumChangelogMode changelogMode;

/**
* Returns a builder to build {@link RowDataDebeziumDeserializeSchema}.
*/
/** Returns a builder to build {@link RowDataDebeziumDeserializeSchema}. */
public static Builder newBuilder() {
return new Builder();
}
Expand Down Expand Up @@ -133,7 +121,9 @@ public static Builder newBuilder() {
public void deserialize(SourceRecord record, Collector<RowData> out) throws Exception {
Envelope.Operation op = Envelope.operationFor(record);
// filter ddl statement to avoid NPE
if (op == null) return;
if (op == null) {
return;
}
Struct value = (Struct) record.value();
Schema valueSchema = record.valueSchema();
if (op == Envelope.Operation.CREATE || op == Envelope.Operation.READ) {
Expand Down Expand Up @@ -193,9 +183,7 @@ public TypeInformation<RowData> getProducedType() {
// Builder
// -------------------------------------------------------------------------------------

/**
* Builder of {@link RowDataDebeziumDeserializeSchema}.
*/
/** Builder of {@link RowDataDebeziumDeserializeSchema}. */
public static class Builder {
private RowType physicalRowType;
private TypeInformation<RowData> resultTypeInfo;
Expand Down Expand Up @@ -258,9 +246,7 @@ public RowDataDebeziumDeserializeSchema build() {
// Runtime Converters
// -------------------------------------------------------------------------------------

/**
* Creates a runtime converter which is null safe.
*/
/** Creates a runtime converter which is null safe. */
private static DeserializationRuntimeConverter createConverter(
LogicalType type,
ZoneId serverTimeZone,
Expand All @@ -275,9 +261,7 @@ private static DeserializationRuntimeConverter createConverter(
// SerializedLambdas (MSHADE-260).
// --------------------------------------------------------------------------------

/**
* Creates a runtime converter which assuming input object is not null.
*/
/** Creates a runtime converter which assuming input object is not null. */
public static DeserializationRuntimeConverter createNotNullConverter(
LogicalType type,
ZoneId serverTimeZone,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -273,6 +273,8 @@ public void setSnapshotHooks(SnapshotPhaseHooks snapshotHooks) {
interface RecordEmitterSupplier<T> extends Serializable {

RecordEmitter<SourceRecords, T, MySqlSplitState> get(
SourceReaderContext sourceReaderContext, MySqlSourceReaderMetrics metrics, MySqlSourceConfig sourceConfig);
SourceReaderContext sourceReaderContext,
MySqlSourceReaderMetrics metrics,
MySqlSourceConfig sourceConfig);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.flink.cdc.connectors.mysql.source.assigners;

import io.debezium.relational.TableId;
import org.apache.flink.cdc.connectors.mysql.source.assigners.state.BinlogPendingSplitsState;
import org.apache.flink.cdc.connectors.mysql.source.assigners.state.PendingSplitsState;
import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig;
Expand All @@ -27,6 +26,8 @@
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit;
import org.apache.flink.util.CollectionUtil;

import io.debezium.relational.TableId;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
Expand Down Expand Up @@ -78,7 +79,8 @@ public boolean waitingForFinishedSplits() {
}

@Override
public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos(boolean isScanNewlyAddedTableEnabled) {
public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos(
boolean isScanNewlyAddedTableEnabled) {
return Collections.emptyList();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,8 @@ public boolean waitingForFinishedSplits() {
}

@Override
public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos(boolean isScanNewlyAddedTableEnabled) {
public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos(
boolean isScanNewlyAddedTableEnabled) {
return snapshotSplitAssigner.getFinishedSplitInfos(isScanNewlyAddedTableEnabled);
}

Expand All @@ -159,7 +160,7 @@ public void addSplits(Collection<MySqlSplit> splits) {
}

@Override
public void addAlreadyProcessedTables(TableId tableId){
public void addAlreadyProcessedTables(TableId tableId) {
snapshotSplitAssigner.addAlreadyProcessedTables(tableId);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -372,7 +372,8 @@ public boolean waitingForFinishedSplits() {
}

@Override
public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos(boolean isScanNewlyAddedTableEnabled) {
public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos(
boolean isScanNewlyAddedTableEnabled) {
if (waitingForFinishedSplits()) {
LOG.error(
"The assigner is not ready to offer finished split information, this should not be called");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,15 @@

package org.apache.flink.cdc.connectors.mysql.source.assigners;

import io.debezium.relational.TableId;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.cdc.common.annotation.Internal;
import org.apache.flink.cdc.connectors.mysql.source.assigners.state.PendingSplitsState;
import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset;
import org.apache.flink.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo;
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit;

import io.debezium.relational.TableId;

import java.util.Collection;
import java.util.List;
import java.util.Map;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.flink.cdc.connectors.mysql.source.enumerator;

import io.debezium.relational.TableId;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.api.connector.source.SourceEvent;
import org.apache.flink.api.connector.source.SplitEnumerator;
Expand All @@ -26,17 +25,31 @@
import org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlSplitAssigner;
import org.apache.flink.cdc.connectors.mysql.source.assigners.state.PendingSplitsState;
import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig;
import org.apache.flink.cdc.connectors.mysql.source.events.*;
import org.apache.flink.cdc.connectors.mysql.source.events.BinlogNewAddedTableEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitAssignedEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitMetaEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitMetaRequestEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitStartEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitSuspendEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitUpdateAckEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.FinishedBinlogNewTableAddRequestEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsAckEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsReportEvent;
import org.apache.flink.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsRequestEvent;
import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset;
import org.apache.flink.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo;
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplit;
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit;
import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
import org.apache.flink.util.FlinkRuntimeException;

import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;

import io.debezium.relational.TableId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
Expand Down Expand Up @@ -66,11 +79,9 @@ public class MySqlSourceEnumerator implements SplitEnumerator<MySqlSplit, Pendin
private final TreeSet<Integer> readersAwaitingSplit;
private List<List<FinishedSnapshotSplitInfo>> binlogSplitMeta;

@Nullable
private Integer binlogSplitTaskId;
@Nullable private Integer binlogSplitTaskId;
private boolean isRuntimeTableAdded = true;


public MySqlSourceEnumerator(
SplitEnumeratorContext<MySqlSplit> context,
MySqlSourceConfig sourceConfig,
Expand Down Expand Up @@ -162,9 +173,11 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
BinlogNewAddedTableEvent event = (BinlogNewAddedTableEvent) sourceEvent;
TableId tableId = new TableId(event.getCatalog(), null, event.getTable());
LOG.info(
"The enumerator receives binlog side new added table from subtask {}, db name: {}, " +
"table name: {}. ",
subtaskId, tableId.catalog(), tableId.table());
"The enumerator receives binlog side new added table from subtask {}, db name: {}, "
+ "table name: {}. ",
subtaskId,
tableId.catalog(),
tableId.table());
splitAssigner.addAlreadyProcessedTables(tableId);
isRuntimeTableAdded = false;
binlogSplitTaskId = subtaskId;
Expand All @@ -180,7 +193,8 @@ public PendingSplitsState snapshotState(long checkpointId) {
public void notifyCheckpointComplete(long checkpointId) {
splitAssigner.notifyCheckpointComplete(checkpointId);
if (!isRuntimeTableAdded) {
context.sendEventToSourceReader(binlogSplitTaskId, new FinishedBinlogNewTableAddRequestEvent());
context.sendEventToSourceReader(
binlogSplitTaskId, new FinishedBinlogNewTableAddRequestEvent());
isRuntimeTableAdded = true;
}
// binlog split may be available after checkpoint complete
Expand Down Expand Up @@ -277,13 +291,15 @@ private void notifyBinlogSplitStart() {
subtaskId);
// get increment new added tables infos when NEWLY_ADDED_ASSIGNING_SNAPSHOT_FINISHED
final List<FinishedSnapshotSplitInfo> finishedSnapshotSplitInfos =
splitAssigner.getFinishedSplitInfos(sourceConfig.isScanNewlyAddedTableEnabled());
splitAssigner.getFinishedSplitInfos(
sourceConfig.isScanNewlyAddedTableEnabled());
context.sendEventToSourceReader(
subtaskId,
new BinlogSplitStartEvent(
finishedSnapshotSplitInfos.size(), finishedSnapshotSplitInfos.stream()
.map(FinishedSnapshotSplitInfo::serialize)
.collect(Collectors.toList())));
finishedSnapshotSplitInfos.size(),
finishedSnapshotSplitInfos.stream()
.map(FinishedSnapshotSplitInfo::serialize)
.collect(Collectors.toList())));
}
}
}
Expand All @@ -292,7 +308,8 @@ private void sendBinlogMeta(int subTask, BinlogSplitMetaRequestEvent requestEven
// initialize once
if (binlogSplitMeta == null) {
final List<FinishedSnapshotSplitInfo> finishedSnapshotSplitInfos =
splitAssigner.getFinishedSplitInfos(sourceConfig.isScanNewlyAddedTableEnabled());
splitAssigner.getFinishedSplitInfos(
sourceConfig.isScanNewlyAddedTableEnabled());
if (finishedSnapshotSplitInfos.isEmpty()) {
LOG.error(
"The assigner offers empty finished split information, this should not happen");
Expand All @@ -305,7 +322,10 @@ private void sendBinlogMeta(int subTask, BinlogSplitMetaRequestEvent requestEven
}
final int requestMetaGroupId = requestEvent.getRequestMetaGroupId();
final int totalFinishedSplitSizeOfReader = requestEvent.getTotalFinishedSplitSize();
final int totalFinishedSplitSizeOfEnumerator = splitAssigner.getFinishedSplitInfos(sourceConfig.isScanNewlyAddedTableEnabled()).size();
final int totalFinishedSplitSizeOfEnumerator =
splitAssigner
.getFinishedSplitInfos(sourceConfig.isScanNewlyAddedTableEnabled())
.size();
if (totalFinishedSplitSizeOfReader > totalFinishedSplitSizeOfEnumerator) {
LOG.warn(
"Total finished split size of subtask {} is {}, while total finished split size of Enumerator is only {}. Try to truncate it",
Expand Down Expand Up @@ -340,5 +360,4 @@ private void sendBinlogMeta(int subTask, BinlogSplitMetaRequestEvent requestEven
totalFinishedSplitSizeOfEnumerator));
}
}

}
Original file line number Diff line number Diff line change
@@ -1,7 +1,30 @@
/*
* 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 org.apache.flink.cdc.connectors.mysql.source.events;

import org.apache.flink.api.connector.source.SourceEvent;
import org.apache.flink.cdc.connectors.mysql.source.enumerator.MySqlSourceEnumerator;
import org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReader;

/**
* The {@link SourceEvent} that {@link MySqlSourceReader} sends to {@link MySqlSourceEnumerator} to
* notify enumerator that sourceReader received new create table.
*/
public class BinlogNewAddedTableEvent implements SourceEvent {

private static final long serialVersionUID = 1L;
Expand All @@ -27,5 +50,4 @@ public String getSchema() {
public String getTable() {
return table;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplit;

import javax.annotation.Nullable;

import java.util.List;

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import org.apache.flink.api.connector.source.SourceEvent;
import org.apache.flink.cdc.connectors.mysql.source.enumerator.MySqlSourceEnumerator;
import org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReader;
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplit;

/**
* The {@link SourceEvent} that {@link MySqlSourceReader} sends to {@link MySqlSourceEnumerator} to
Expand Down
Loading

0 comments on commit 0dca9c7

Please sign in to comment.