[oracle] Introduce SCAN_INCREMENTAL_SNAPSHOT_ENABLED option for OracleTableSourceFactory
parent
a94eefd90a
commit
94fa8b347f
@ -1,50 +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.ververica.cdc.connectors.oracle.source.meta.events;
|
|
||||||
|
|
||||||
import org.apache.flink.api.connector.source.SourceEvent;
|
|
||||||
|
|
||||||
import com.ververica.cdc.connectors.base.source.JdbcIncrementalSource;
|
|
||||||
import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The {@link SourceEvent} that {@link IncrementalSourceEnumerator} sends to {@link
|
|
||||||
* JdbcIncrementalSource} to pass the latest finished snapshot splits size.
|
|
||||||
*/
|
|
||||||
public class LatestFinishedSplitsSizeEvent implements SourceEvent {
|
|
||||||
|
|
||||||
private static final long serialVersionUID = 1L;
|
|
||||||
private final int latestFinishedSplitsSize;
|
|
||||||
|
|
||||||
public LatestFinishedSplitsSizeEvent(int latestFinishedSplitsSize) {
|
|
||||||
this.latestFinishedSplitsSize = latestFinishedSplitsSize;
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getLatestFinishedSplitsSize() {
|
|
||||||
return latestFinishedSplitsSize;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "LatestFinishedSplitsSizeEvent{"
|
|
||||||
+ "latestFinishedSplitsSize="
|
|
||||||
+ latestFinishedSplitsSize
|
|
||||||
+ '}';
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,151 +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.ververica.cdc.connectors.oracle.source.meta.split;
|
|
||||||
|
|
||||||
import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo;
|
|
||||||
import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset;
|
|
||||||
import io.debezium.relational.TableId;
|
|
||||||
import io.debezium.relational.history.TableChanges;
|
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Objects;
|
|
||||||
|
|
||||||
/** The split to describe the redo log of Oracle table(s). */
|
|
||||||
public class OracleRedoLogSplit extends OracleSplit {
|
|
||||||
|
|
||||||
private final RedoLogOffset startingOffset;
|
|
||||||
private final RedoLogOffset endingOffset;
|
|
||||||
private final List<FinishedSnapshotSplitInfo> finishedSnapshotSplitInfos;
|
|
||||||
private final Map<TableId, TableChanges.TableChange> tableSchemas;
|
|
||||||
private final int totalFinishedSplitSize;
|
|
||||||
private final boolean isSuspended;
|
|
||||||
@Nullable transient byte[] serializedFormCache;
|
|
||||||
|
|
||||||
public OracleRedoLogSplit(
|
|
||||||
String splitId,
|
|
||||||
RedoLogOffset startingOffset,
|
|
||||||
RedoLogOffset endingOffset,
|
|
||||||
List<FinishedSnapshotSplitInfo> finishedSnapshotSplitInfos,
|
|
||||||
Map<TableId, TableChanges.TableChange> tableSchemas,
|
|
||||||
int totalFinishedSplitSize,
|
|
||||||
boolean isSuspended) {
|
|
||||||
super(splitId);
|
|
||||||
this.startingOffset = startingOffset;
|
|
||||||
this.endingOffset = endingOffset;
|
|
||||||
this.finishedSnapshotSplitInfos = finishedSnapshotSplitInfos;
|
|
||||||
this.tableSchemas = tableSchemas;
|
|
||||||
this.totalFinishedSplitSize = totalFinishedSplitSize;
|
|
||||||
this.isSuspended = isSuspended;
|
|
||||||
}
|
|
||||||
|
|
||||||
public OracleRedoLogSplit(
|
|
||||||
String splitId,
|
|
||||||
RedoLogOffset startingOffset,
|
|
||||||
RedoLogOffset endingOffset,
|
|
||||||
List<FinishedSnapshotSplitInfo> finishedSnapshotSplitInfos,
|
|
||||||
Map<TableId, TableChanges.TableChange> tableSchemas,
|
|
||||||
int totalFinishedSplitSize) {
|
|
||||||
super(splitId);
|
|
||||||
this.startingOffset = startingOffset;
|
|
||||||
this.endingOffset = endingOffset;
|
|
||||||
this.finishedSnapshotSplitInfos = finishedSnapshotSplitInfos;
|
|
||||||
this.tableSchemas = tableSchemas;
|
|
||||||
this.totalFinishedSplitSize = totalFinishedSplitSize;
|
|
||||||
this.isSuspended = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
public RedoLogOffset getStartingOffset() {
|
|
||||||
return startingOffset;
|
|
||||||
}
|
|
||||||
|
|
||||||
public RedoLogOffset getEndingOffset() {
|
|
||||||
return endingOffset;
|
|
||||||
}
|
|
||||||
|
|
||||||
public List<FinishedSnapshotSplitInfo> getFinishedSnapshotSplitInfos() {
|
|
||||||
return finishedSnapshotSplitInfos;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Map<TableId, TableChanges.TableChange> getTableSchemas() {
|
|
||||||
return tableSchemas;
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getTotalFinishedSplitSize() {
|
|
||||||
return totalFinishedSplitSize;
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean isSuspended() {
|
|
||||||
return isSuspended;
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean isCompletedSplit() {
|
|
||||||
return totalFinishedSplitSize == finishedSnapshotSplitInfos.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
if (this == o) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
if (!(o instanceof OracleRedoLogSplit)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (!super.equals(o)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
OracleRedoLogSplit that = (OracleRedoLogSplit) o;
|
|
||||||
return totalFinishedSplitSize == that.totalFinishedSplitSize
|
|
||||||
&& isSuspended == that.isSuspended
|
|
||||||
&& Objects.equals(startingOffset, that.startingOffset)
|
|
||||||
&& Objects.equals(endingOffset, that.endingOffset)
|
|
||||||
&& Objects.equals(finishedSnapshotSplitInfos, that.finishedSnapshotSplitInfos)
|
|
||||||
&& Objects.equals(tableSchemas, that.tableSchemas);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
return Objects.hash(
|
|
||||||
super.hashCode(),
|
|
||||||
startingOffset,
|
|
||||||
endingOffset,
|
|
||||||
finishedSnapshotSplitInfos,
|
|
||||||
tableSchemas,
|
|
||||||
totalFinishedSplitSize,
|
|
||||||
isSuspended);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "OracleRedoLogSplit{"
|
|
||||||
+ "splitId='"
|
|
||||||
+ splitId
|
|
||||||
+ '\''
|
|
||||||
+ ", offset="
|
|
||||||
+ startingOffset
|
|
||||||
+ ", endOffset="
|
|
||||||
+ endingOffset
|
|
||||||
+ ", isSuspended="
|
|
||||||
+ isSuspended
|
|
||||||
+ '}';
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,147 +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.ververica.cdc.connectors.oracle.source.meta.split;
|
|
||||||
|
|
||||||
import org.apache.flink.table.types.logical.RowType;
|
|
||||||
|
|
||||||
import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset;
|
|
||||||
import io.debezium.relational.TableId;
|
|
||||||
import io.debezium.relational.history.TableChanges;
|
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
/** The split to describe a split of an Oracle table snapshot. */
|
|
||||||
public class OracleSnapshotSplit extends OracleSplit {
|
|
||||||
|
|
||||||
private final TableId tableId;
|
|
||||||
private final RowType splitKeyType;
|
|
||||||
private final Map<TableId, TableChanges.TableChange> tableSchemas;
|
|
||||||
|
|
||||||
@Nullable private final Object[] splitStart;
|
|
||||||
@Nullable private final Object[] splitEnd;
|
|
||||||
/** The high watermark is not bull when the split read finished. */
|
|
||||||
@Nullable private final RedoLogOffset highWatermark;
|
|
||||||
|
|
||||||
@Nullable transient byte[] serializedFormCache;
|
|
||||||
|
|
||||||
public OracleSnapshotSplit(
|
|
||||||
String splitId,
|
|
||||||
TableId tableId,
|
|
||||||
RowType splitKeyType,
|
|
||||||
Map<TableId, TableChanges.TableChange> tableSchemas,
|
|
||||||
@Nullable Object[] splitStart,
|
|
||||||
@Nullable Object[] splitEnd,
|
|
||||||
@Nullable RedoLogOffset highWatermark) {
|
|
||||||
super(splitId);
|
|
||||||
this.tableId = tableId;
|
|
||||||
this.splitKeyType = splitKeyType;
|
|
||||||
this.tableSchemas = tableSchemas;
|
|
||||||
this.splitStart = splitStart;
|
|
||||||
this.splitEnd = splitEnd;
|
|
||||||
this.highWatermark = highWatermark;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Map<TableId, TableChanges.TableChange> getTableSchemas() {
|
|
||||||
return tableSchemas;
|
|
||||||
}
|
|
||||||
|
|
||||||
public TableId getTableId() {
|
|
||||||
return tableId;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
public Object[] getSplitStart() {
|
|
||||||
return splitStart;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
public Object[] getSplitEnd() {
|
|
||||||
return splitEnd;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
public RedoLogOffset getHighWatermark() {
|
|
||||||
return highWatermark;
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean isSnapshotReadFinished() {
|
|
||||||
return highWatermark != null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
if (this == o) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
if (o == null || getClass() != o.getClass()) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (!super.equals(o)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
OracleSnapshotSplit that = (OracleSnapshotSplit) o;
|
|
||||||
return Objects.equals(tableId, that.tableId)
|
|
||||||
&& Objects.equals(splitKeyType, that.splitKeyType)
|
|
||||||
&& Arrays.equals(splitStart, that.splitStart)
|
|
||||||
&& Arrays.equals(splitEnd, that.splitEnd)
|
|
||||||
&& Objects.equals(highWatermark, that.highWatermark);
|
|
||||||
}
|
|
||||||
|
|
||||||
public RowType getSplitKeyType() {
|
|
||||||
return splitKeyType;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
int result = Objects.hash(super.hashCode(), tableId, splitKeyType, highWatermark);
|
|
||||||
result = 31 * result + Arrays.hashCode(splitStart);
|
|
||||||
result = 31 * result + Arrays.hashCode(splitEnd);
|
|
||||||
result = 31 * result + Arrays.hashCode(serializedFormCache);
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
String splitKeyTypeSummary =
|
|
||||||
splitKeyType.getFields().stream()
|
|
||||||
.map(RowType.RowField::asSummaryString)
|
|
||||||
.collect(Collectors.joining(",", "[", "]"));
|
|
||||||
return "OracleSnapshotSplit{"
|
|
||||||
+ "tableId="
|
|
||||||
+ tableId
|
|
||||||
+ ", splitId='"
|
|
||||||
+ splitId
|
|
||||||
+ '\''
|
|
||||||
+ ", splitKeyType="
|
|
||||||
+ splitKeyTypeSummary
|
|
||||||
+ ", splitStart="
|
|
||||||
+ Arrays.toString(splitStart)
|
|
||||||
+ ", splitEnd="
|
|
||||||
+ Arrays.toString(splitEnd)
|
|
||||||
+ ", highWatermark="
|
|
||||||
+ highWatermark
|
|
||||||
+ '}';
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,198 +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.ververica.cdc.connectors.oracle.source.meta.split;
|
|
||||||
//
|
|
||||||
// import org.apache.flink.core.memory.DataInputDeserializer;
|
|
||||||
// import org.apache.flink.core.memory.DataOutputSerializer;
|
|
||||||
// import org.apache.flink.table.types.logical.RowType;
|
|
||||||
// import org.apache.flink.table.types.logical.utils.LogicalTypeParser;
|
|
||||||
//
|
|
||||||
// import com.ververica.cdc.connectors.base.source.meta.offset.Offset;
|
|
||||||
// import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory;
|
|
||||||
// import com.ververica.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo;
|
|
||||||
// import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit;
|
|
||||||
// import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase;
|
|
||||||
// import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitSerializer;
|
|
||||||
// import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit;
|
|
||||||
// import com.ververica.cdc.connectors.base.utils.SerializerUtils;
|
|
||||||
// import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffsetFactory;
|
|
||||||
// import com.ververica.cdc.debezium.history.FlinkJsonTableChangeSerializer;
|
|
||||||
// import io.debezium.document.Document;
|
|
||||||
// import io.debezium.document.DocumentReader;
|
|
||||||
// import io.debezium.relational.TableId;
|
|
||||||
// import io.debezium.relational.history.TableChanges;
|
|
||||||
//
|
|
||||||
// import java.io.IOException;
|
|
||||||
// import java.nio.charset.StandardCharsets;
|
|
||||||
// import java.util.ArrayList;
|
|
||||||
// import java.util.HashMap;
|
|
||||||
// import java.util.List;
|
|
||||||
// import java.util.Map;
|
|
||||||
//
|
|
||||||
/// ** An Oracle serializer for the {@link SourceSplitBase}. */
|
|
||||||
// public class OracleSourceSplitSerializer extends SourceSplitSerializer {
|
|
||||||
//
|
|
||||||
// private static final int SNAPSHOT_SPLIT_FLAG = 1;
|
|
||||||
// private static final int REDOLOG_SPLIT_FLAG = 2;
|
|
||||||
// RedoLogOffsetFactory offsetFactory;
|
|
||||||
//
|
|
||||||
// public OracleSourceSplitSerializer(RedoLogOffsetFactory offsetFactory) {
|
|
||||||
// this.offsetFactory = offsetFactory;
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public OffsetFactory getOffsetFactory() {
|
|
||||||
// return offsetFactory;
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public Offset readOffsetPosition(int offsetVersion, DataInputDeserializer in)
|
|
||||||
// throws IOException {
|
|
||||||
// return super.readOffsetPosition(offsetVersion, in);
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public Offset readOffsetPosition(DataInputDeserializer in) throws IOException {
|
|
||||||
// return super.readOffsetPosition(in);
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public void writeOffsetPosition(Offset offset, DataOutputSerializer out) throws IOException {
|
|
||||||
// super.writeOffsetPosition(offset, out);
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public OffsetDeserializer createOffsetDeserializer() {
|
|
||||||
// return super.createOffsetDeserializer();
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public FinishedSnapshotSplitInfo deserialize(byte[] serialized) {
|
|
||||||
// return super.deserialize(serialized);
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public byte[] serialize(SourceSplitBase split) throws IOException {
|
|
||||||
// return super.serialize(split);
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public SourceSplitBase deserialize(int version, byte[] serialized) throws IOException {
|
|
||||||
// return super.deserialize(version, serialized);
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Override
|
|
||||||
// public SourceSplitBase deserializeSplit(int version, byte[] serialized) throws IOException {
|
|
||||||
// final DataInputDeserializer in = new DataInputDeserializer(serialized);
|
|
||||||
//
|
|
||||||
// int splitKind = in.readInt();
|
|
||||||
// if (splitKind == SNAPSHOT_SPLIT_FLAG) {
|
|
||||||
// TableId tableId = TableId.parse(in.readUTF(), false);
|
|
||||||
// String splitId = in.readUTF();
|
|
||||||
// RowType splitKeyType = (RowType) LogicalTypeParser.parse(in.readUTF());
|
|
||||||
// Object[] splitBoundaryStart = SerializerUtils.serializedStringToRow(in.readUTF());
|
|
||||||
// Object[] splitBoundaryEnd = SerializerUtils.serializedStringToRow(in.readUTF());
|
|
||||||
// Offset highWatermark = readOffsetPosition(version, in);
|
|
||||||
// Map<TableId, TableChanges.TableChange> tableSchemas = readTableSchemas(version, in);
|
|
||||||
//
|
|
||||||
// return new SnapshotSplit(
|
|
||||||
// tableId,
|
|
||||||
// splitId,
|
|
||||||
// splitKeyType,
|
|
||||||
// splitBoundaryStart,
|
|
||||||
// splitBoundaryEnd,
|
|
||||||
// highWatermark,
|
|
||||||
// tableSchemas);
|
|
||||||
// } else if (splitKind == REDOLOG_SPLIT_FLAG) {
|
|
||||||
// String splitId = in.readUTF();
|
|
||||||
// // skip split Key Type
|
|
||||||
// in.readUTF();
|
|
||||||
// Offset startingOffset = readOffsetPosition(version, in);
|
|
||||||
// Offset endingOffset = readOffsetPosition(version, in);
|
|
||||||
// List<FinishedSnapshotSplitInfo> finishedSplitsInfo =
|
|
||||||
// readFinishedSplitsInfo(version, in);
|
|
||||||
// Map<TableId, TableChanges.TableChange> tableChangeMap = readTableSchemas(version, in);
|
|
||||||
// int totalFinishedSplitSize = finishedSplitsInfo.size();
|
|
||||||
// if (version == 3) {
|
|
||||||
// totalFinishedSplitSize = in.readInt();
|
|
||||||
// }
|
|
||||||
// in.releaseArrays();
|
|
||||||
// return new StreamSplit(
|
|
||||||
// splitId,
|
|
||||||
// startingOffset,
|
|
||||||
// endingOffset,
|
|
||||||
// finishedSplitsInfo,
|
|
||||||
// tableChangeMap,
|
|
||||||
// totalFinishedSplitSize);
|
|
||||||
// } else {
|
|
||||||
// throw new IOException("Unknown split kind: " + splitKind);
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// private List<FinishedSnapshotSplitInfo> readFinishedSplitsInfo(
|
|
||||||
// int version, DataInputDeserializer in) throws IOException {
|
|
||||||
// List<FinishedSnapshotSplitInfo> finishedSplitsInfo = new ArrayList<>();
|
|
||||||
// final int size = in.readInt();
|
|
||||||
// for (int i = 0; i < size; i++) {
|
|
||||||
// TableId tableId = TableId.parse(in.readUTF(), false);
|
|
||||||
// String splitId = in.readUTF();
|
|
||||||
// Object[] splitStart = SerializerUtils.serializedStringToRow(in.readUTF());
|
|
||||||
// Object[] splitEnd = SerializerUtils.serializedStringToRow(in.readUTF());
|
|
||||||
// OffsetFactory offsetFactory =
|
|
||||||
// (OffsetFactory) SerializerUtils.serializedStringToObject(in.readUTF());
|
|
||||||
// Offset highWatermark = readOffsetPosition(version, in);
|
|
||||||
//
|
|
||||||
// finishedSplitsInfo.add(
|
|
||||||
// new FinishedSnapshotSplitInfo(
|
|
||||||
// tableId, splitId, splitStart, splitEnd, highWatermark,
|
|
||||||
// offsetFactory));
|
|
||||||
// }
|
|
||||||
// return finishedSplitsInfo;
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// private static Map<TableId, TableChanges.TableChange> readTableSchemas(
|
|
||||||
// int version, DataInputDeserializer in) throws IOException {
|
|
||||||
// DocumentReader documentReader = DocumentReader.defaultReader();
|
|
||||||
// Map<TableId, TableChanges.TableChange> tableSchemas = new HashMap<>();
|
|
||||||
// final int size = in.readInt();
|
|
||||||
// for (int i = 0; i < size; i++) {
|
|
||||||
// TableId tableId = TableId.parse(in.readUTF(), false);
|
|
||||||
// final String tableChangeStr;
|
|
||||||
// switch (version) {
|
|
||||||
// case 1:
|
|
||||||
// tableChangeStr = in.readUTF();
|
|
||||||
// break;
|
|
||||||
// case 2:
|
|
||||||
// case 3:
|
|
||||||
// final int len = in.readInt();
|
|
||||||
// final byte[] bytes = new byte[len];
|
|
||||||
// in.read(bytes);
|
|
||||||
// tableChangeStr = new String(bytes, StandardCharsets.UTF_8);
|
|
||||||
// break;
|
|
||||||
// default:
|
|
||||||
// throw new IOException("Unknown version: " + version);
|
|
||||||
// }
|
|
||||||
// Document document = documentReader.read(tableChangeStr);
|
|
||||||
// TableChanges.TableChange tableChange =
|
|
||||||
// FlinkJsonTableChangeSerializer.fromDocument(document, false);
|
|
||||||
// tableSchemas.put(tableId, tableChange);
|
|
||||||
// }
|
|
||||||
// return tableSchemas;
|
|
||||||
// }
|
|
||||||
// }
|
|
@ -1,81 +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.ververica.cdc.connectors.oracle.source.meta.split;
|
|
||||||
|
|
||||||
import org.apache.flink.api.connector.source.SourceSplit;
|
|
||||||
|
|
||||||
import io.debezium.relational.TableId;
|
|
||||||
import io.debezium.relational.history.TableChanges;
|
|
||||||
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Objects;
|
|
||||||
|
|
||||||
/** The split of table comes from a Table that splits by primary key. */
|
|
||||||
public abstract class OracleSplit implements SourceSplit {
|
|
||||||
|
|
||||||
protected final String splitId;
|
|
||||||
|
|
||||||
public OracleSplit(String splitId) {
|
|
||||||
this.splitId = splitId;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String splitId() {
|
|
||||||
return splitId;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Checks whether this split is a snapshot split. */
|
|
||||||
public final boolean isSnapshotSplit() {
|
|
||||||
return getClass() == OracleSnapshotSplit.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Checks whether this split is a redo log split. */
|
|
||||||
public final boolean isBinlogSplit() {
|
|
||||||
return getClass() == OracleRedoLogSplit.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Casts this split into a {@link OracleSnapshotSplit}. */
|
|
||||||
public final OracleSnapshotSplit asSnapshotSplit() {
|
|
||||||
return (OracleSnapshotSplit) this;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Casts this split into a {@link OracleRedoLogSplit}. */
|
|
||||||
public final OracleRedoLogSplit asRedoLogSplit() {
|
|
||||||
return (OracleRedoLogSplit) this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public abstract Map<TableId, TableChanges.TableChange> getTableSchemas();
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
if (this == o) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
if (o == null || getClass() != o.getClass()) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
OracleSplit that = (OracleSplit) o;
|
|
||||||
return Objects.equals(splitId, that.splitId);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
return Objects.hash(splitId);
|
|
||||||
}
|
|
||||||
}
|
|
@ -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.ververica.cdc.connectors.oracle.source.table;
|
|
||||||
|
|
||||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
|
||||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
|
||||||
import org.apache.flink.table.connector.ChangelogMode;
|
|
||||||
import org.apache.flink.table.connector.source.DynamicTableSource;
|
|
||||||
import org.apache.flink.table.connector.source.ScanTableSource;
|
|
||||||
import org.apache.flink.table.connector.source.SourceProvider;
|
|
||||||
import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
|
|
||||||
import org.apache.flink.table.data.RowData;
|
|
||||||
import org.apache.flink.table.types.DataType;
|
|
||||||
import org.apache.flink.table.types.logical.RowType;
|
|
||||||
import org.apache.flink.types.RowKind;
|
|
||||||
|
|
||||||
import com.ververica.cdc.connectors.base.source.JdbcIncrementalSource;
|
|
||||||
import com.ververica.cdc.connectors.oracle.source.OracleSourceBuilder;
|
|
||||||
import com.ververica.cdc.connectors.oracle.table.OracleDeserializationConverterFactory;
|
|
||||||
import com.ververica.cdc.connectors.oracle.table.OracleReadableMetaData;
|
|
||||||
import com.ververica.cdc.connectors.oracle.table.StartupOptions;
|
|
||||||
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
|
|
||||||
import com.ververica.cdc.debezium.table.MetadataConverter;
|
|
||||||
import com.ververica.cdc.debezium.table.RowDataDebeziumDeserializeSchema;
|
|
||||||
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Properties;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
import java.util.stream.Stream;
|
|
||||||
|
|
||||||
import static org.apache.flink.util.Preconditions.checkNotNull;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A {@link DynamicTableSource} that describes how to create a Oracle binlog from a logical
|
|
||||||
* description.
|
|
||||||
*/
|
|
||||||
public class OracleTableSource implements ScanTableSource, SupportsReadingMetadata {
|
|
||||||
|
|
||||||
private final ResolvedSchema physicalSchema;
|
|
||||||
private final int port;
|
|
||||||
private final String hostname;
|
|
||||||
private final String database;
|
|
||||||
private final String username;
|
|
||||||
private final String password;
|
|
||||||
private final String tableName;
|
|
||||||
private final String schemaName;
|
|
||||||
private final Properties dbzProperties;
|
|
||||||
private final StartupOptions startupOptions;
|
|
||||||
|
|
||||||
// --------------------------------------------------------------------------------------------
|
|
||||||
// Mutable attributes
|
|
||||||
// --------------------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
/** Data type that describes the final output of the source. */
|
|
||||||
protected DataType producedDataType;
|
|
||||||
|
|
||||||
/** Metadata that is appended at the end of a physical source row. */
|
|
||||||
protected List<String> metadataKeys;
|
|
||||||
|
|
||||||
public OracleTableSource(
|
|
||||||
ResolvedSchema physicalSchema,
|
|
||||||
int port,
|
|
||||||
String hostname,
|
|
||||||
String database,
|
|
||||||
String tableName,
|
|
||||||
String schemaName,
|
|
||||||
String username,
|
|
||||||
String password,
|
|
||||||
Properties dbzProperties,
|
|
||||||
StartupOptions startupOptions) {
|
|
||||||
this.physicalSchema = physicalSchema;
|
|
||||||
this.port = port;
|
|
||||||
this.hostname = checkNotNull(hostname);
|
|
||||||
this.database = checkNotNull(database);
|
|
||||||
this.tableName = checkNotNull(tableName);
|
|
||||||
this.schemaName = checkNotNull(schemaName);
|
|
||||||
this.username = checkNotNull(username);
|
|
||||||
this.password = checkNotNull(password);
|
|
||||||
this.dbzProperties = dbzProperties;
|
|
||||||
this.startupOptions = startupOptions;
|
|
||||||
this.producedDataType = physicalSchema.toPhysicalRowDataType();
|
|
||||||
this.metadataKeys = Collections.emptyList();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ChangelogMode getChangelogMode() {
|
|
||||||
return ChangelogMode.newBuilder()
|
|
||||||
.addContainedKind(RowKind.INSERT)
|
|
||||||
.addContainedKind(RowKind.UPDATE_BEFORE)
|
|
||||||
.addContainedKind(RowKind.UPDATE_AFTER)
|
|
||||||
.addContainedKind(RowKind.DELETE)
|
|
||||||
.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
|
|
||||||
|
|
||||||
RowType physicalDataType =
|
|
||||||
(RowType) physicalSchema.toPhysicalRowDataType().getLogicalType();
|
|
||||||
MetadataConverter[] metadataConverters = getMetadataConverters();
|
|
||||||
TypeInformation<RowData> typeInfo = scanContext.createTypeInformation(producedDataType);
|
|
||||||
|
|
||||||
DebeziumDeserializationSchema<RowData> deserializer =
|
|
||||||
RowDataDebeziumDeserializeSchema.newBuilder()
|
|
||||||
.setPhysicalRowType(physicalDataType)
|
|
||||||
.setMetadataConverters(metadataConverters)
|
|
||||||
.setResultTypeInfo(typeInfo)
|
|
||||||
.setUserDefinedConverterFactory(
|
|
||||||
OracleDeserializationConverterFactory.instance())
|
|
||||||
.build();
|
|
||||||
JdbcIncrementalSource<RowData> oracleChangeEventSource =
|
|
||||||
new <RowData>OracleSourceBuilder()
|
|
||||||
.hostname(hostname)
|
|
||||||
.port(port)
|
|
||||||
.databaseList(database)
|
|
||||||
.schemaList(schemaName)
|
|
||||||
.tableList(schemaName + "." + tableName)
|
|
||||||
.username(username)
|
|
||||||
.password(password)
|
|
||||||
.deserializer(deserializer)
|
|
||||||
.includeSchemaChanges(true)
|
|
||||||
.debeziumProperties(dbzProperties)
|
|
||||||
.build();
|
|
||||||
|
|
||||||
return SourceProvider.of(oracleChangeEventSource);
|
|
||||||
}
|
|
||||||
|
|
||||||
private MetadataConverter[] getMetadataConverters() {
|
|
||||||
if (metadataKeys.isEmpty()) {
|
|
||||||
return new MetadataConverter[0];
|
|
||||||
}
|
|
||||||
|
|
||||||
return metadataKeys.stream()
|
|
||||||
.map(
|
|
||||||
key ->
|
|
||||||
Stream.of(OracleReadableMetaData.values())
|
|
||||||
.filter(m -> m.getKey().equals(key))
|
|
||||||
.findFirst()
|
|
||||||
.orElseThrow(IllegalStateException::new))
|
|
||||||
.map(OracleReadableMetaData::getConverter)
|
|
||||||
.toArray(MetadataConverter[]::new);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public DynamicTableSource copy() {
|
|
||||||
OracleTableSource source =
|
|
||||||
new OracleTableSource(
|
|
||||||
physicalSchema,
|
|
||||||
port,
|
|
||||||
hostname,
|
|
||||||
database,
|
|
||||||
tableName,
|
|
||||||
schemaName,
|
|
||||||
username,
|
|
||||||
password,
|
|
||||||
dbzProperties,
|
|
||||||
startupOptions);
|
|
||||||
source.metadataKeys = metadataKeys;
|
|
||||||
source.producedDataType = producedDataType;
|
|
||||||
return source;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
if (this == o) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
if (o == null || getClass() != o.getClass()) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
OracleTableSource that = (OracleTableSource) o;
|
|
||||||
return port == that.port
|
|
||||||
&& Objects.equals(physicalSchema, that.physicalSchema)
|
|
||||||
&& Objects.equals(hostname, that.hostname)
|
|
||||||
&& Objects.equals(database, that.database)
|
|
||||||
&& Objects.equals(username, that.username)
|
|
||||||
&& Objects.equals(password, that.password)
|
|
||||||
&& Objects.equals(tableName, that.tableName)
|
|
||||||
&& Objects.equals(schemaName, that.schemaName)
|
|
||||||
&& Objects.equals(dbzProperties, that.dbzProperties)
|
|
||||||
&& Objects.equals(startupOptions, that.startupOptions)
|
|
||||||
&& Objects.equals(producedDataType, that.producedDataType)
|
|
||||||
&& Objects.equals(metadataKeys, that.metadataKeys);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
return Objects.hash(
|
|
||||||
physicalSchema,
|
|
||||||
port,
|
|
||||||
hostname,
|
|
||||||
database,
|
|
||||||
username,
|
|
||||||
password,
|
|
||||||
tableName,
|
|
||||||
schemaName,
|
|
||||||
dbzProperties,
|
|
||||||
startupOptions,
|
|
||||||
producedDataType,
|
|
||||||
metadataKeys);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String asSummaryString() {
|
|
||||||
return "Oracle-CDC";
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Map<String, DataType> listReadableMetadata() {
|
|
||||||
return Stream.of(OracleReadableMetaData.values())
|
|
||||||
.collect(
|
|
||||||
Collectors.toMap(
|
|
||||||
OracleReadableMetaData::getKey,
|
|
||||||
OracleReadableMetaData::getDataType));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void applyReadableMetadata(List<String> metadataKeys, DataType producedDataType) {
|
|
||||||
this.metadataKeys = metadataKeys;
|
|
||||||
this.producedDataType = producedDataType;
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,174 +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.ververica.cdc.connectors.oracle.source.table;
|
|
||||||
|
|
||||||
import org.apache.flink.configuration.ConfigOption;
|
|
||||||
import org.apache.flink.configuration.ConfigOptions;
|
|
||||||
import org.apache.flink.configuration.ReadableConfig;
|
|
||||||
import org.apache.flink.table.api.ValidationException;
|
|
||||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
|
||||||
import org.apache.flink.table.connector.source.DynamicTableSource;
|
|
||||||
import org.apache.flink.table.factories.DynamicTableSourceFactory;
|
|
||||||
import org.apache.flink.table.factories.FactoryUtil;
|
|
||||||
|
|
||||||
import com.ververica.cdc.connectors.oracle.table.StartupOptions;
|
|
||||||
import com.ververica.cdc.debezium.table.DebeziumOptions;
|
|
||||||
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
import static com.ververica.cdc.debezium.table.DebeziumOptions.getDebeziumProperties;
|
|
||||||
|
|
||||||
/** Factory for creating configured instance of {@link OracleTableSource}. */
|
|
||||||
public class OracleTableSourceFactory implements DynamicTableSourceFactory {
|
|
||||||
|
|
||||||
private static final String IDENTIFIER = "oracle-cdc-new";
|
|
||||||
|
|
||||||
private static final ConfigOption<String> HOSTNAME =
|
|
||||||
ConfigOptions.key("hostname")
|
|
||||||
.stringType()
|
|
||||||
.noDefaultValue()
|
|
||||||
.withDescription("IP address or hostname of the Oracle database server.");
|
|
||||||
|
|
||||||
private static final ConfigOption<Integer> PORT =
|
|
||||||
ConfigOptions.key("port")
|
|
||||||
.intType()
|
|
||||||
.defaultValue(1521)
|
|
||||||
.withDescription("Integer port number of the Oracle database server.");
|
|
||||||
|
|
||||||
private static final ConfigOption<String> USERNAME =
|
|
||||||
ConfigOptions.key("username")
|
|
||||||
.stringType()
|
|
||||||
.noDefaultValue()
|
|
||||||
.withDescription(
|
|
||||||
"Name of the Oracle database to use when connecting to the Oracle database server.");
|
|
||||||
|
|
||||||
private static final ConfigOption<String> PASSWORD =
|
|
||||||
ConfigOptions.key("password")
|
|
||||||
.stringType()
|
|
||||||
.noDefaultValue()
|
|
||||||
.withDescription(
|
|
||||||
"Password to use when connecting to the oracle database server.");
|
|
||||||
|
|
||||||
private static final ConfigOption<String> DATABASE_NAME =
|
|
||||||
ConfigOptions.key("database-name")
|
|
||||||
.stringType()
|
|
||||||
.noDefaultValue()
|
|
||||||
.withDescription("Database name of the Oracle server to monitor.");
|
|
||||||
|
|
||||||
private static final ConfigOption<String> SCHEMA_NAME =
|
|
||||||
ConfigOptions.key("schema-name")
|
|
||||||
.stringType()
|
|
||||||
.noDefaultValue()
|
|
||||||
.withDescription("Schema name of the Oracle database to monitor.");
|
|
||||||
|
|
||||||
private static final ConfigOption<String> TABLE_NAME =
|
|
||||||
ConfigOptions.key("table-name")
|
|
||||||
.stringType()
|
|
||||||
.noDefaultValue()
|
|
||||||
.withDescription("Table name of the Oracle database to monitor.");
|
|
||||||
|
|
||||||
public static final ConfigOption<String> SCAN_STARTUP_MODE =
|
|
||||||
ConfigOptions.key("scan.startup.mode")
|
|
||||||
.stringType()
|
|
||||||
.defaultValue("initial")
|
|
||||||
.withDescription(
|
|
||||||
"Optional startup mode for Oracle CDC consumer, valid enumerations are "
|
|
||||||
+ "\"initial\", \"latest-offset\"");
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public DynamicTableSource createDynamicTableSource(Context context) {
|
|
||||||
final FactoryUtil.TableFactoryHelper helper =
|
|
||||||
FactoryUtil.createTableFactoryHelper(this, context);
|
|
||||||
helper.validateExcept(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX);
|
|
||||||
|
|
||||||
final ReadableConfig config = helper.getOptions();
|
|
||||||
String hostname = config.get(HOSTNAME);
|
|
||||||
String username = config.get(USERNAME);
|
|
||||||
String password = config.get(PASSWORD);
|
|
||||||
String databaseName = config.get(DATABASE_NAME);
|
|
||||||
String tableName = config.get(TABLE_NAME);
|
|
||||||
String schemaName = config.get(SCHEMA_NAME);
|
|
||||||
int port = config.get(PORT);
|
|
||||||
StartupOptions startupOptions = getStartupOptions(config);
|
|
||||||
ResolvedSchema physicalSchema = context.getCatalogTable().getResolvedSchema();
|
|
||||||
|
|
||||||
return new OracleTableSource(
|
|
||||||
physicalSchema,
|
|
||||||
port,
|
|
||||||
hostname,
|
|
||||||
databaseName,
|
|
||||||
tableName,
|
|
||||||
schemaName,
|
|
||||||
username,
|
|
||||||
password,
|
|
||||||
getDebeziumProperties(context.getCatalogTable().getOptions()),
|
|
||||||
startupOptions);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String factoryIdentifier() {
|
|
||||||
return IDENTIFIER;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Set<ConfigOption<?>> requiredOptions() {
|
|
||||||
Set<ConfigOption<?>> options = new HashSet<>();
|
|
||||||
options.add(HOSTNAME);
|
|
||||||
options.add(USERNAME);
|
|
||||||
options.add(PASSWORD);
|
|
||||||
options.add(DATABASE_NAME);
|
|
||||||
options.add(TABLE_NAME);
|
|
||||||
options.add(SCHEMA_NAME);
|
|
||||||
return options;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Set<ConfigOption<?>> optionalOptions() {
|
|
||||||
Set<ConfigOption<?>> options = new HashSet<>();
|
|
||||||
options.add(PORT);
|
|
||||||
options.add(SCAN_STARTUP_MODE);
|
|
||||||
|
|
||||||
return options;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial";
|
|
||||||
private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset";
|
|
||||||
|
|
||||||
private static StartupOptions getStartupOptions(ReadableConfig config) {
|
|
||||||
String modeString = config.get(SCAN_STARTUP_MODE);
|
|
||||||
|
|
||||||
switch (modeString.toLowerCase()) {
|
|
||||||
case SCAN_STARTUP_MODE_VALUE_INITIAL:
|
|
||||||
return StartupOptions.initial();
|
|
||||||
|
|
||||||
case SCAN_STARTUP_MODE_VALUE_LATEST:
|
|
||||||
return StartupOptions.latest();
|
|
||||||
|
|
||||||
default:
|
|
||||||
throw new ValidationException(
|
|
||||||
String.format(
|
|
||||||
"Invalid value for option '%s'. Supported values are [%s, %s], but was: %s",
|
|
||||||
SCAN_STARTUP_MODE.key(),
|
|
||||||
SCAN_STARTUP_MODE_VALUE_INITIAL,
|
|
||||||
SCAN_STARTUP_MODE_VALUE_LATEST,
|
|
||||||
modeString));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,125 +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.ververica.cdc.connectors.oracle.source.utils;
|
|
||||||
//
|
|
||||||
// import org.apache.flink.core.memory.DataInputDeserializer;
|
|
||||||
// import org.apache.flink.core.memory.DataOutputSerializer;
|
|
||||||
//
|
|
||||||
// import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset;
|
|
||||||
// import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffsetSerializer;
|
|
||||||
// import io.debezium.DebeziumException;
|
|
||||||
// import io.debezium.util.HexConverter;
|
|
||||||
//
|
|
||||||
// import java.io.ByteArrayInputStream;
|
|
||||||
// import java.io.ByteArrayOutputStream;
|
|
||||||
// import java.io.IOException;
|
|
||||||
// import java.io.ObjectInputStream;
|
|
||||||
// import java.io.ObjectOutputStream;
|
|
||||||
//
|
|
||||||
/// ** Utils for serialization and deserialization. */
|
|
||||||
// public class SerializerUtils {
|
|
||||||
//
|
|
||||||
// private SerializerUtils() {}
|
|
||||||
//
|
|
||||||
// public static void writeRedoLogPosition(RedoLogOffset offset, DataOutputSerializer out)
|
|
||||||
// throws IOException {
|
|
||||||
// out.writeBoolean(offset != null);
|
|
||||||
// if (offset != null) {
|
|
||||||
// byte[] redoLogOffsetBytes = RedoLogOffsetSerializer.INSTANCE.serialize(offset);
|
|
||||||
// out.writeInt(redoLogOffsetBytes.length);
|
|
||||||
// out.write(redoLogOffsetBytes);
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// public static RedoLogOffset readRedoLogPosition(int offsetVersion, DataInputDeserializer in)
|
|
||||||
// throws IOException {
|
|
||||||
// switch (offsetVersion) {
|
|
||||||
// case 1:
|
|
||||||
//// return in.readBoolean() ? new RedoLogOffset(in.readUTF(), in.readLong()) : null;
|
|
||||||
// case 2:
|
|
||||||
// case 3:
|
|
||||||
// case 4:
|
|
||||||
// return readRedoLogPosition(in);
|
|
||||||
// default:
|
|
||||||
// throw new IOException("Unknown version: " + offsetVersion);
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// public static RedoLogOffset readRedoLogPosition(DataInputDeserializer in) throws IOException {
|
|
||||||
// boolean offsetNonNull = in.readBoolean();
|
|
||||||
// if (offsetNonNull) {
|
|
||||||
// int redoLogOffsetBytesLength = in.readInt();
|
|
||||||
// byte[] redoLogOffsetBytes = new byte[redoLogOffsetBytesLength];
|
|
||||||
// in.readFully(redoLogOffsetBytes);
|
|
||||||
// return RedoLogOffsetSerializer.INSTANCE.deserialize(redoLogOffsetBytes);
|
|
||||||
// } else {
|
|
||||||
// return null;
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// public static String rowToSerializedString(Object[] splitBoundary) {
|
|
||||||
// try (final ByteArrayOutputStream bos = new ByteArrayOutputStream();
|
|
||||||
// ObjectOutputStream oos = new ObjectOutputStream(bos)) {
|
|
||||||
// oos.writeObject(splitBoundary);
|
|
||||||
// return HexConverter.convertToHexString(bos.toByteArray());
|
|
||||||
// } catch (IOException e) {
|
|
||||||
// throw new DebeziumException(
|
|
||||||
// String.format("Cannot serialize split boundary information %s",
|
|
||||||
// splitBoundary));
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// public static String rowToSerializedString(Object splitBoundary) {
|
|
||||||
// try (final ByteArrayOutputStream bos = new ByteArrayOutputStream();
|
|
||||||
// ObjectOutputStream oos = new ObjectOutputStream(bos)) {
|
|
||||||
// oos.writeObject(splitBoundary);
|
|
||||||
// return HexConverter.convertToHexString(bos.toByteArray());
|
|
||||||
// } catch (IOException e) {
|
|
||||||
// throw new DebeziumException(
|
|
||||||
// String.format("Cannot serialize split boundary information %s",
|
|
||||||
// splitBoundary));
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// public static Object[] serializedStringToRow(String serialized) {
|
|
||||||
// try (final ByteArrayInputStream bis =
|
|
||||||
// new ByteArrayInputStream(HexConverter.convertFromHex(serialized));
|
|
||||||
// ObjectInputStream ois = new ObjectInputStream(bis)) {
|
|
||||||
// return (Object[]) ois.readObject();
|
|
||||||
// } catch (Exception e) {
|
|
||||||
// throw new DebeziumException(
|
|
||||||
// String.format(
|
|
||||||
// "Failed to deserialize split boundary with value '%s'", serialized),
|
|
||||||
// e);
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// public static Object serializedStringToObject(String serialized) {
|
|
||||||
// try (final ByteArrayInputStream bis =
|
|
||||||
// new ByteArrayInputStream(HexConverter.convertFromHex(serialized));
|
|
||||||
// ObjectInputStream ois = new ObjectInputStream(bis)) {
|
|
||||||
// return ois.readObject();
|
|
||||||
// } catch (Exception e) {
|
|
||||||
// throw new DebeziumException(
|
|
||||||
// String.format(
|
|
||||||
// "Failed to deserialize split boundary with value '%s'", serialized),
|
|
||||||
// e);
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
// }
|
|
Loading…
Reference in New Issue