[FLINK-35243][cdc-common] Extends more schema change event types support
This close #3521.pull/3559/head
parent
d3473de4db
commit
78fda8b5ab
@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.common.event;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.PublicEvolving;
|
||||
import org.apache.flink.cdc.common.source.DataSource;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A {@link SchemaChangeEvent} that represents an {@code DROP TABLE} DDL. this will be sent by
|
||||
* {@link DataSource} before all {@link DataChangeEvent} with the same tableId.
|
||||
*/
|
||||
@PublicEvolving
|
||||
public class DropTableEvent implements SchemaChangeEvent {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final TableId tableId;
|
||||
|
||||
public DropTableEvent(TableId tableId) {
|
||||
this.tableId = tableId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof DropTableEvent)) {
|
||||
return false;
|
||||
}
|
||||
DropTableEvent that = (DropTableEvent) o;
|
||||
return Objects.equals(tableId, that.tableId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(tableId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DropTableEvent{" + "tableId=" + tableId + '}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableId tableId() {
|
||||
return tableId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SchemaChangeEventType getType() {
|
||||
return SchemaChangeEventType.DROP_TABLE;
|
||||
}
|
||||
}
|
@ -0,0 +1,37 @@
|
||||
/*
|
||||
* 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.common.event;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.PublicEvolving;
|
||||
import org.apache.flink.cdc.common.schema.Schema;
|
||||
|
||||
/** A {@link SchemaChangeEvent} that supports appending schema before change event. */
|
||||
@PublicEvolving
|
||||
public interface SchemaChangeEventWithPreSchema extends SchemaChangeEvent {
|
||||
|
||||
/** Describes if this event already has schema before change info. */
|
||||
boolean hasPreSchema();
|
||||
|
||||
/** Append schema before change info to this event. */
|
||||
void fillPreSchema(Schema oldSchema);
|
||||
|
||||
/** Check if this event contains redundant schema change request only. */
|
||||
default boolean trimRedundantChanges() {
|
||||
return false;
|
||||
}
|
||||
}
|
@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.common.event;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.PublicEvolving;
|
||||
import org.apache.flink.cdc.common.source.DataSource;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A {@link SchemaChangeEvent} that represents an {@code TRUNCATE TABLE} DDL. this will be sent by
|
||||
* {@link DataSource} before all {@link DataChangeEvent} with the same tableId.
|
||||
*/
|
||||
@PublicEvolving
|
||||
public class TruncateTableEvent implements SchemaChangeEvent {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final TableId tableId;
|
||||
|
||||
public TruncateTableEvent(TableId tableId) {
|
||||
this.tableId = tableId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof TruncateTableEvent)) {
|
||||
return false;
|
||||
}
|
||||
TruncateTableEvent that = (TruncateTableEvent) o;
|
||||
return Objects.equals(tableId, that.tableId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(tableId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "TruncateTableEvent{" + "tableId=" + tableId + '}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableId tableId() {
|
||||
return tableId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SchemaChangeEventType getType() {
|
||||
return SchemaChangeEventType.TRUNCATE_TABLE;
|
||||
}
|
||||
}
|
@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.common.event.visitor;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.Internal;
|
||||
import org.apache.flink.cdc.common.event.AddColumnEvent;
|
||||
|
||||
/** Visitor for {@link AddColumnEvent}s. */
|
||||
@Internal
|
||||
@FunctionalInterface
|
||||
public interface AddColumnEventVisitor<T, E extends Throwable> {
|
||||
T visit(AddColumnEvent event) throws E;
|
||||
}
|
@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.common.event.visitor;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.Internal;
|
||||
import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
|
||||
|
||||
/** Visitor for {@link AlterColumnTypeEvent}s. */
|
||||
@Internal
|
||||
@FunctionalInterface
|
||||
public interface AlterColumnTypeEventVisitor<T, E extends Throwable> {
|
||||
T visit(AlterColumnTypeEvent event) throws E;
|
||||
}
|
@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.common.event.visitor;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.Internal;
|
||||
import org.apache.flink.cdc.common.event.CreateTableEvent;
|
||||
|
||||
/** Visitor for {@link CreateTableEvent}s. */
|
||||
@Internal
|
||||
@FunctionalInterface
|
||||
public interface CreateTableEventVisitor<T, E extends Throwable> {
|
||||
T visit(CreateTableEvent event) throws E;
|
||||
}
|
@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.common.event.visitor;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.Internal;
|
||||
import org.apache.flink.cdc.common.event.DropColumnEvent;
|
||||
|
||||
/** Visitor for {@link DropColumnEvent}s. */
|
||||
@Internal
|
||||
@FunctionalInterface
|
||||
public interface DropColumnEventVisitor<T, E extends Throwable> {
|
||||
T visit(DropColumnEvent event) throws E;
|
||||
}
|
@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.common.event.visitor;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.Internal;
|
||||
import org.apache.flink.cdc.common.event.DropTableEvent;
|
||||
|
||||
/** Visitor for {@link DropTableEvent}s. */
|
||||
@Internal
|
||||
@FunctionalInterface
|
||||
public interface DropTableEventVisitor<T, E extends Throwable> {
|
||||
T visit(DropTableEvent event) throws E;
|
||||
}
|
@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.common.event.visitor;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.Internal;
|
||||
import org.apache.flink.cdc.common.event.RenameColumnEvent;
|
||||
|
||||
/** Visitor for {@link RenameColumnEvent}s. */
|
||||
@Internal
|
||||
@FunctionalInterface
|
||||
public interface RenameColumnEventVisitor<T, E extends Throwable> {
|
||||
T visit(RenameColumnEvent event) throws E;
|
||||
}
|
@ -0,0 +1,83 @@
|
||||
/*
|
||||
* 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.common.event.visitor;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.Internal;
|
||||
import org.apache.flink.cdc.common.event.AddColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
|
||||
import org.apache.flink.cdc.common.event.CreateTableEvent;
|
||||
import org.apache.flink.cdc.common.event.DropColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.DropTableEvent;
|
||||
import org.apache.flink.cdc.common.event.RenameColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.TruncateTableEvent;
|
||||
|
||||
/** Visitor clas for all {@link SchemaChangeEvent}s and returns a specific typed object. */
|
||||
@Internal
|
||||
public class SchemaChangeEventVisitor {
|
||||
public static <T, E extends Throwable> T visit(
|
||||
SchemaChangeEvent event,
|
||||
AddColumnEventVisitor<T, E> addColumnVisitor,
|
||||
AlterColumnTypeEventVisitor<T, E> alterColumnTypeEventVisitor,
|
||||
CreateTableEventVisitor<T, E> createTableEventVisitor,
|
||||
DropColumnEventVisitor<T, E> dropColumnEventVisitor,
|
||||
DropTableEventVisitor<T, E> dropTableEventVisitor,
|
||||
RenameColumnEventVisitor<T, E> renameColumnEventVisitor,
|
||||
TruncateTableEventVisitor<T, E> truncateTableEventVisitor)
|
||||
throws E {
|
||||
if (event instanceof AddColumnEvent) {
|
||||
if (addColumnVisitor == null) {
|
||||
return null;
|
||||
}
|
||||
return addColumnVisitor.visit((AddColumnEvent) event);
|
||||
} else if (event instanceof AlterColumnTypeEvent) {
|
||||
if (alterColumnTypeEventVisitor == null) {
|
||||
return null;
|
||||
}
|
||||
return alterColumnTypeEventVisitor.visit((AlterColumnTypeEvent) event);
|
||||
} else if (event instanceof CreateTableEvent) {
|
||||
if (createTableEventVisitor == null) {
|
||||
return null;
|
||||
}
|
||||
return createTableEventVisitor.visit((CreateTableEvent) event);
|
||||
} else if (event instanceof DropColumnEvent) {
|
||||
if (dropColumnEventVisitor == null) {
|
||||
return null;
|
||||
}
|
||||
return dropColumnEventVisitor.visit((DropColumnEvent) event);
|
||||
} else if (event instanceof DropTableEvent) {
|
||||
if (dropTableEventVisitor == null) {
|
||||
return null;
|
||||
}
|
||||
return dropTableEventVisitor.visit((DropTableEvent) event);
|
||||
} else if (event instanceof RenameColumnEvent) {
|
||||
if (renameColumnEventVisitor == null) {
|
||||
return null;
|
||||
}
|
||||
return renameColumnEventVisitor.visit((RenameColumnEvent) event);
|
||||
} else if (event instanceof TruncateTableEvent) {
|
||||
if (truncateTableEventVisitor == null) {
|
||||
return null;
|
||||
}
|
||||
return truncateTableEventVisitor.visit((TruncateTableEvent) event);
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Unknown schema change event type " + event.getType());
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.common.event.visitor;
|
||||
|
||||
import org.apache.flink.cdc.common.annotation.Internal;
|
||||
import org.apache.flink.cdc.common.event.TruncateTableEvent;
|
||||
|
||||
/** Visitor for {@link TruncateTableEvent}s. */
|
||||
@Internal
|
||||
@FunctionalInterface
|
||||
public interface TruncateTableEventVisitor<T, E extends Throwable> {
|
||||
T visit(TruncateTableEvent event) throws E;
|
||||
}
|
@ -0,0 +1,108 @@
|
||||
/*
|
||||
* 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.runtime.serializer.event;
|
||||
|
||||
import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
|
||||
import org.apache.flink.api.common.typeutils.TypeSerializer;
|
||||
import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
|
||||
import org.apache.flink.cdc.common.event.DropTableEvent;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.types.DataType;
|
||||
import org.apache.flink.cdc.runtime.serializer.MapSerializer;
|
||||
import org.apache.flink.cdc.runtime.serializer.StringSerializer;
|
||||
import org.apache.flink.cdc.runtime.serializer.TableIdSerializer;
|
||||
import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton;
|
||||
import org.apache.flink.cdc.runtime.serializer.schema.DataTypeSerializer;
|
||||
import org.apache.flink.core.memory.DataInputView;
|
||||
import org.apache.flink.core.memory.DataOutputView;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/** A {@link TypeSerializer} for {@link DropTableEvent}. */
|
||||
public class DropTableEventSerializer extends TypeSerializerSingleton<DropTableEvent> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/** Sharable instance of the TableIdSerializer. */
|
||||
public static final DropTableEventSerializer INSTANCE = new DropTableEventSerializer();
|
||||
|
||||
private final TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE;
|
||||
private final MapSerializer<String, DataType> typeMapSerializer =
|
||||
new MapSerializer<>(StringSerializer.INSTANCE, new DataTypeSerializer());
|
||||
|
||||
@Override
|
||||
public boolean isImmutableType() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DropTableEvent createInstance() {
|
||||
return new DropTableEvent(TableId.tableId("unknown"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DropTableEvent copy(DropTableEvent from) {
|
||||
return new DropTableEvent(from.tableId());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DropTableEvent copy(DropTableEvent from, DropTableEvent reuse) {
|
||||
return copy(from);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength() {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serialize(DropTableEvent record, DataOutputView target) throws IOException {
|
||||
tableIdSerializer.serialize(record.tableId(), target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DropTableEvent deserialize(DataInputView source) throws IOException {
|
||||
return new DropTableEvent(tableIdSerializer.deserialize(source));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DropTableEvent deserialize(DropTableEvent reuse, DataInputView source)
|
||||
throws IOException {
|
||||
return deserialize(source);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(DataInputView source, DataOutputView target) throws IOException {
|
||||
serialize(deserialize(source), target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeSerializerSnapshot<DropTableEvent> snapshotConfiguration() {
|
||||
return new DropTableEventSerializer.DropTableEventSerializerSnapshot();
|
||||
}
|
||||
|
||||
/** Serializer configuration snapshot for compatibility and format evolution. */
|
||||
@SuppressWarnings("WeakerAccess")
|
||||
public static final class DropTableEventSerializerSnapshot
|
||||
extends SimpleTypeSerializerSnapshot<DropTableEvent> {
|
||||
|
||||
public DropTableEventSerializerSnapshot() {
|
||||
super(() -> INSTANCE);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,108 @@
|
||||
/*
|
||||
* 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.runtime.serializer.event;
|
||||
|
||||
import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
|
||||
import org.apache.flink.api.common.typeutils.TypeSerializer;
|
||||
import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.event.TruncateTableEvent;
|
||||
import org.apache.flink.cdc.common.types.DataType;
|
||||
import org.apache.flink.cdc.runtime.serializer.MapSerializer;
|
||||
import org.apache.flink.cdc.runtime.serializer.StringSerializer;
|
||||
import org.apache.flink.cdc.runtime.serializer.TableIdSerializer;
|
||||
import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton;
|
||||
import org.apache.flink.cdc.runtime.serializer.schema.DataTypeSerializer;
|
||||
import org.apache.flink.core.memory.DataInputView;
|
||||
import org.apache.flink.core.memory.DataOutputView;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/** A {@link TypeSerializer} for {@link TruncateTableEvent}. */
|
||||
public class TruncateTableEventSerializer extends TypeSerializerSingleton<TruncateTableEvent> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/** Sharable instance of the TableIdSerializer. */
|
||||
public static final TruncateTableEventSerializer INSTANCE = new TruncateTableEventSerializer();
|
||||
|
||||
private final TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE;
|
||||
private final MapSerializer<String, DataType> typeMapSerializer =
|
||||
new MapSerializer<>(StringSerializer.INSTANCE, new DataTypeSerializer());
|
||||
|
||||
@Override
|
||||
public boolean isImmutableType() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TruncateTableEvent createInstance() {
|
||||
return new TruncateTableEvent(TableId.tableId("unknown"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public TruncateTableEvent copy(TruncateTableEvent from) {
|
||||
return new TruncateTableEvent(from.tableId());
|
||||
}
|
||||
|
||||
@Override
|
||||
public TruncateTableEvent copy(TruncateTableEvent from, TruncateTableEvent reuse) {
|
||||
return copy(from);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength() {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serialize(TruncateTableEvent record, DataOutputView target) throws IOException {
|
||||
tableIdSerializer.serialize(record.tableId(), target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TruncateTableEvent deserialize(DataInputView source) throws IOException {
|
||||
return new TruncateTableEvent(tableIdSerializer.deserialize(source));
|
||||
}
|
||||
|
||||
@Override
|
||||
public TruncateTableEvent deserialize(TruncateTableEvent reuse, DataInputView source)
|
||||
throws IOException {
|
||||
return deserialize(source);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(DataInputView source, DataOutputView target) throws IOException {
|
||||
serialize(deserialize(source), target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeSerializerSnapshot<TruncateTableEvent> snapshotConfiguration() {
|
||||
return new TruncateTableEventSerializer.TruncateTableEventSerializerSnapshot();
|
||||
}
|
||||
|
||||
/** Serializer configuration snapshot for compatibility and format evolution. */
|
||||
@SuppressWarnings("WeakerAccess")
|
||||
public static final class TruncateTableEventSerializerSnapshot
|
||||
extends SimpleTypeSerializerSnapshot<TruncateTableEvent> {
|
||||
|
||||
public TruncateTableEventSerializerSnapshot() {
|
||||
super(() -> INSTANCE);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,50 @@
|
||||
/*
|
||||
* 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.runtime.serializer.event;
|
||||
|
||||
import org.apache.flink.api.common.typeutils.TypeSerializer;
|
||||
import org.apache.flink.cdc.common.event.DropTableEvent;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.runtime.serializer.SerializerTestBase;
|
||||
|
||||
/** A test for the {@link RenameTableEventSerializer}. */
|
||||
public class DropTableEventSerializerTest extends SerializerTestBase<DropTableEvent> {
|
||||
@Override
|
||||
protected TypeSerializer<DropTableEvent> createSerializer() {
|
||||
return DropTableEventSerializer.INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getLength() {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<DropTableEvent> getTypeClass() {
|
||||
return DropTableEvent.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DropTableEvent[] getTestData() {
|
||||
return new DropTableEvent[] {
|
||||
new DropTableEvent(TableId.tableId("table")),
|
||||
new DropTableEvent(TableId.tableId("schema", "table")),
|
||||
new DropTableEvent(TableId.tableId("namespace", "schema", "table"))
|
||||
};
|
||||
}
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.runtime.serializer.event;
|
||||
|
||||
import org.apache.flink.api.common.typeutils.TypeSerializer;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.event.TruncateTableEvent;
|
||||
import org.apache.flink.cdc.runtime.serializer.SerializerTestBase;
|
||||
|
||||
/** A test for the {@link RenameTableEventSerializer}. */
|
||||
public class TruncateTableEventSerializerTest extends SerializerTestBase<TruncateTableEvent> {
|
||||
@Override
|
||||
protected TypeSerializer<TruncateTableEvent> createSerializer() {
|
||||
|
||||
return TruncateTableEventSerializer.INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getLength() {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<TruncateTableEvent> getTypeClass() {
|
||||
return TruncateTableEvent.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TruncateTableEvent[] getTestData() {
|
||||
return new TruncateTableEvent[] {
|
||||
new TruncateTableEvent(TableId.tableId("table")),
|
||||
new TruncateTableEvent(TableId.tableId("schema", "table")),
|
||||
new TruncateTableEvent(TableId.tableId("namespace", "schema", "table"))
|
||||
};
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue