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] [base] fix TableId.java use identifier decode error with four quota #2443

Merged
merged 3 commits into from
Sep 7, 2023
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 @@ -70,7 +70,7 @@ public byte[] serialize(SourceSplitBase split) throws IOException {
boolean useCatalogBeforeSchema =
SerializerUtils.shouldUseCatalogBeforeSchema(snapshotSplit.getTableId());
out.writeBoolean(useCatalogBeforeSchema);
out.writeUTF(snapshotSplit.getTableId().toString());
out.writeUTF(snapshotSplit.getTableId().toDoubleQuotedString());
out.writeUTF(snapshotSplit.splitId());
out.writeUTF(snapshotSplit.getSplitKeyType().asSerializableString());

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/*
* Copyright 2023 Ververica Inc.
*
* 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.ververica.cdc.connectors.base.source.meta.split;

import org.apache.flink.table.types.logical.BigIntType;
import org.apache.flink.table.types.logical.RowType;

import com.ververica.cdc.connectors.base.source.meta.offset.Offset;
import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory;
import io.debezium.relational.TableId;
import org.junit.Test;

import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;

import static org.junit.Assert.assertEquals;

/** Tests for {@link SourceSplitSerializer}. */
public class SourceSplitSerializerTest {

@Test
public void testSnapshotTableIdSerializeAndDeserialize() throws IOException {
SnapshotSplit snapshotSplitBefore =
new SnapshotSplit(
new TableId("cata`log\"", "s\"che`ma", "ta\"ble.1`"),
"test",
new RowType(
Collections.singletonList(
new RowType.RowField("id", new BigIntType()))),
null,
null,
null,
new HashMap<>());

SourceSplitSerializer sourceSplitSerializer =
new SourceSplitSerializer() {
@Override
public OffsetFactory getOffsetFactory() {
return new OffsetFactory() {
@Override
public Offset newOffset(Map<String, String> offset) {
return null;
}

@Override
public Offset newOffset(String filename, Long position) {
return null;
}

@Override
public Offset newOffset(Long position) {
return null;
}

@Override
public Offset createTimestampOffset(long timestampMillis) {
return null;
}

@Override
public Offset createInitialOffset() {
return null;
}

@Override
public Offset createNoStoppingOffset() {
return null;
}
};
}
};

SnapshotSplit snapshotSplitAfter =
(SnapshotSplit)
sourceSplitSerializer.deserialize(
sourceSplitSerializer.getVersion(),
sourceSplitSerializer.serialize(snapshotSplitBefore));

assertEquals(snapshotSplitBefore.getTableId(), snapshotSplitAfter.getTableId());
}
}
Loading