From 1de7805f405efc3d0e8a7ff1964f72f14a306afa Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:24:25 +0800 Subject: [PATCH 01/20] a. Pulsar Table Source implementation code --- .../table/source/PulsarReadableMetadata.java | 142 +++++++++++ .../table/source/PulsarRowDataConverter.java | 134 ++++++++++ .../PulsarTableDeserializationSchema.java | 115 +++++++++ ...lsarTableDeserializationSchemaFactory.java | 215 ++++++++++++++++ .../table/source/PulsarTableSource.java | 229 ++++++++++++++++++ 5 files changed, 835 insertions(+) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java new file mode 100644 index 0000000000000..b769db3488bd7 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java @@ -0,0 +1,142 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.Message; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Class for reading metadata fields from a Pulsar message and put in corresponding Flink row + * fields. + * + *

Contains list of readable metadata and provide util methods for metadata manipulation. + */ +public class PulsarReadableMetadata implements Serializable { + + private static final long serialVersionUID = -4409932324481235973L; + + private final List connectorMetadataKeys; + + private final List metadataConverters; + + public PulsarReadableMetadata(List connectorMetadataKeys) { + this.connectorMetadataKeys = connectorMetadataKeys; + this.metadataConverters = initializeMetadataConverters(); + } + + private List initializeMetadataConverters() { + return connectorMetadataKeys.stream() + .map( + k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .map(m -> m.converter) + .collect(Collectors.toList()); + } + + public void appendProducedRowWithMetadata( + GenericRowData producedRowData, int physicalArity, Message message) { + for (int metadataPos = 0; metadataPos < metadataConverters.size(); metadataPos++) { + producedRowData.setField( + physicalArity + metadataPos, metadataConverters.get(metadataPos).read(message)); + } + } + + public int getConnectorMetadataArity() { + return metadataConverters.size(); + } + + interface MetadataConverter extends Serializable { + Object read(Message message); + } + + /** Lists the metadata that is readable from a Pulsar message. Used in SQL source connector. */ + public enum ReadableMetadata { + TOPIC( + "topic", + DataTypes.STRING().notNull(), + message -> StringData.fromString(message.getTopicName())), + + MESSAGE_SIZE("message_size", DataTypes.INT().notNull(), message -> message.size()), + + PRODUCER_NAME( + "producer_name", + DataTypes.STRING().notNull(), + message -> StringData.fromString(message.getProducerName())), + + MESSAGE_ID( + "message_id", + DataTypes.BYTES().notNull(), + message -> message.getMessageId().toByteArray()), + + SEQUENCE_ID("sequenceId", DataTypes.BIGINT().notNull(), Message::getSequenceId), + + PUBLISH_TIME( + "publish_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + message -> TimestampData.fromEpochMillis(message.getPublishTime())), + + EVENT_TIME( + "event_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + message -> TimestampData.fromEpochMillis(message.getEventTime())), + + PROPERTIES( + "properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .notNull(), + message -> { + final Map map = new HashMap<>(); + for (Map.Entry e : message.getProperties().entrySet()) { + map.put( + StringData.fromString(e.getKey()), + StringData.fromString(e.getValue())); + } + return new GenericMapData(map); + }); + + public final String key; + + public final DataType dataType; + + public final MetadataConverter converter; + + ReadableMetadata(String key, DataType dataType, MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java new file mode 100644 index 0000000000000..603da1e950b6f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java @@ -0,0 +1,134 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.DeserializationException; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import org.apache.pulsar.client.api.Message; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.List; + +/** + * Contains the projection information needed to map a Pulsar message to proper key fields, value + * fields and metadata fields. + */ +public class PulsarRowDataConverter implements Serializable { + private static final long serialVersionUID = -3399264407634977459L; + + private final int physicalArity; + + private final int[] keyProjection; + + private final int[] valueProjection; + + private final PulsarReadableMetadata readableMetadata; + + private final boolean upsertMode; + + public PulsarRowDataConverter( + int physicalArity, + int[] keyProjection, + int[] valueProjection, + PulsarReadableMetadata readableMetadata, + boolean upsertMode) { + this.physicalArity = physicalArity; + this.keyProjection = keyProjection; + this.valueProjection = valueProjection; + this.readableMetadata = readableMetadata; + this.upsertMode = upsertMode; + } + + public void projectToProducedRowAndCollect( + Message message, + List keyRowDataList, + List valueRowDataList, + Collector collector) { + // no key defined + if (hasNoKeyProjection()) { + valueRowDataList.forEach( + valueRow -> emitRow(null, (GenericRowData) valueRow, collector, message)); + } else { + // otherwise emit a value for each key + valueRowDataList.forEach( + valueRow -> + keyRowDataList.forEach( + keyRow -> + emitRow( + (GenericRowData) keyRow, + (GenericRowData) valueRow, + collector, + message))); + } + } + + public void projectToRowWithNullValueRow( + Message message, List keyRowDataList, Collector collector) { + for (RowData keyRow : keyRowDataList) { + emitRow((GenericRowData) keyRow, null, collector, message); + } + } + + private void emitRow( + @Nullable GenericRowData physicalKeyRow, + @Nullable GenericRowData physicalValueRow, + Collector collector, + Message message) { + + final RowKind rowKind; + if (physicalValueRow == null) { + if (upsertMode) { + rowKind = RowKind.DELETE; + } else { + throw new DeserializationException( + "Invalid null value received in non-upsert mode. Could not to set row kind for output record." + + "upsert mode is not supported yet."); + } + + } else { + rowKind = physicalValueRow.getRowKind(); + } + + final GenericRowData producedRow = + new GenericRowData( + rowKind, physicalArity + readableMetadata.getConnectorMetadataArity()); + + for (int valuePos = 0; valuePos < valueProjection.length; valuePos++) { + producedRow.setField(valueProjection[valuePos], physicalValueRow.getField(valuePos)); + } + + for (int keyPos = 0; keyPos < keyProjection.length; keyPos++) { + assert physicalKeyRow != null; + producedRow.setField(keyProjection[keyPos], physicalKeyRow.getField(keyPos)); + } + + readableMetadata.appendProducedRowWithMetadata(producedRow, physicalArity, message); + collector.collect(producedRow); + } + + private boolean hasNoKeyProjection() { + return keyProjection.length == 0; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java new file mode 100644 index 0000000000000..4edf89f91a440 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java @@ -0,0 +1,115 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.api.common.functions.util.ListCollector; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Collector; + +import org.apache.pulsar.client.api.Message; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A specific {@link PulsarDeserializationSchema} for {@link PulsarTableSource}. + * + *

Both Flink's key decoding format and value decoding format are wrapped in this class. It is + * responsible for getting metadata fields from a physical pulsar message body, and the final + * projection mapping from Pulsar message fields to Flink row. + * + *

After retrieving key and value bytes and convert them into a list of {@link RowData}, it then + * delegates metadata appending, key and value {@link RowData} combining to a {@link + * PulsarRowDataConverter} instance. + */ +public class PulsarTableDeserializationSchema implements PulsarDeserializationSchema { + private static final long serialVersionUID = -3298784447432136216L; + + private final TypeInformation producedTypeInfo; + + @Nullable private final DeserializationSchema keyDeserialization; + + private final DeserializationSchema valueDeserialization; + + private final PulsarRowDataConverter rowDataConverter; + + private final boolean upsertMode; + + public PulsarTableDeserializationSchema( + @Nullable DeserializationSchema keyDeserialization, + DeserializationSchema valueDeserialization, + TypeInformation producedTypeInfo, + PulsarRowDataConverter rowDataConverter, + boolean upsertMode) { + this.keyDeserialization = keyDeserialization; + this.valueDeserialization = checkNotNull(valueDeserialization); + this.rowDataConverter = checkNotNull(rowDataConverter); + this.producedTypeInfo = checkNotNull(producedTypeInfo); + this.upsertMode = upsertMode; + } + + @Override + public void open( + DeserializationSchema.InitializationContext context, SourceConfiguration configuration) + throws Exception { + if (keyDeserialization != null) { + keyDeserialization.open(context); + } + valueDeserialization.open(context); + } + + @Override + public void deserialize(Message message, Collector collector) + throws IOException { + + // Get the key row data + List keyRowData = new ArrayList<>(); + if (keyDeserialization != null) { + keyDeserialization.deserialize(message.getKeyBytes(), new ListCollector<>(keyRowData)); + } + + // Get the value row data + List valueRowData = new ArrayList<>(); + + if (upsertMode && message.getData().length == 0) { + checkNotNull(keyDeserialization, "upsert mode must specify a key format"); + rowDataConverter.projectToRowWithNullValueRow(message, keyRowData, collector); + return; + } + + valueDeserialization.deserialize(message.getData(), new ListCollector<>(valueRowData)); + + rowDataConverter.projectToProducedRowAndCollect( + message, keyRowData, valueRowData, collector); + } + + @Override + public TypeInformation getProducedType() { + return producedTypeInfo; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java new file mode 100644 index 0000000000000..ab9b6a0ba9386 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java @@ -0,0 +1,215 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Contains key, value projection and format information, and use such information to create a + * {@link PulsarTableDeserializationSchema} instance + * used by runtime {@link org.apache.flink.connector.pulsar.source.PulsarSource} instance. + * + *

A Flink row fields has a strict order: Physical Fields (Key + value) + Format Metadata Fields + * Connector Metadata Fields. Physical Fields are fields come directly from Pulsar message body; + * Format Metadata Fields are from the extra information from the decoding format. Connector + * metadata fields are the ones most Pulsar messages have, such as publish time, message size and + * producer name. + * + *

In general, Physical fields + Format Metadata fields are contained in the RowData decoded + * using valueDecodingFormat. Only Connector Metadata fields needs to be appended to the decoded + * RowData. The tricky part is to put format metadata and connector metadata in the right location. + * This requires an explicit adjustment process. + * + *

For example, suppose Physical Fields (Key + value) + Format Metadata Fields + Connector + * Metadata Fields. has arity of 11, key projection is [0, 6], and physical value projection is [1, + * 2, 3, 4, 5], Then after the adjustment, key projection should be [0, 6], physical value + * projection should be [1, 2, 3, 4, 5] and format metadata projection should be [7], connector + * metadata projection should be [8, 9, 10]. + */ +public class PulsarTableDeserializationSchemaFactory implements Serializable { + + private static final long serialVersionUID = 6091562041940740434L; + + private final DataType physicalDataType; + + @Nullable private final DecodingFormat> keyDecodingFormat; + + private final int[] keyProjection; + + private final DecodingFormat> valueDecodingFormat; + + private final int[] valueProjection; + + // -------------------------------------------------------------------------------------------- + // Mutable attributes. Will be updated after the applyReadableMetadata() + // -------------------------------------------------------------------------------------------- + private DataType producedDataType; + + private List connectorMetadataKeys; + + private final boolean upsertMode; + + public PulsarTableDeserializationSchemaFactory( + DataType physicalDataType, + @Nullable DecodingFormat> keyDecodingFormat, + int[] keyProjection, + DecodingFormat> valueDecodingFormat, + int[] valueProjection, + boolean upsertMode) { + this.physicalDataType = + checkNotNull(physicalDataType, "field physicalDataType must not be null."); + this.keyDecodingFormat = keyDecodingFormat; + this.keyProjection = checkNotNull(keyProjection); + this.valueDecodingFormat = + checkNotNull(valueDecodingFormat, "field valueDecodingFormat must not be null."); + this.valueProjection = + checkNotNull(valueProjection, "field valueProjection must not be null."); + + this.producedDataType = physicalDataType; + this.connectorMetadataKeys = Collections.emptyList(); + this.upsertMode = upsertMode; + } + + private @Nullable DeserializationSchema createDeserialization( + DynamicTableSource.Context context, + @Nullable DecodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeDecoder(context, physicalFormatDataType); + } + + public PulsarDeserializationSchema createPulsarDeserialization( + ScanTableSource.ScanContext context) { + final DeserializationSchema keyDeserialization = + createDeserialization(context, keyDecodingFormat, keyProjection, ""); + final DeserializationSchema valueDeserialization = + createDeserialization(context, valueDecodingFormat, valueProjection, ""); + + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + + final PulsarReadableMetadata readableMetadata = + new PulsarReadableMetadata(connectorMetadataKeys); + + // Get Physical Fields (key + value) + Format Metadata arity + final int physicalPlusFormatMetadataArity = + DataType.getFieldDataTypes(producedDataType).size() + - readableMetadata.getConnectorMetadataArity(); + final int[] physicalValuePlusFormatMetadataProjection = + adjustValueProjectionByAppendConnectorMetadata(physicalPlusFormatMetadataArity); + + final PulsarRowDataConverter rowDataConverter = + new PulsarRowDataConverter( + physicalPlusFormatMetadataArity, + keyProjection, + physicalValuePlusFormatMetadataProjection, + readableMetadata, + upsertMode); + + return new PulsarTableDeserializationSchema( + keyDeserialization, + valueDeserialization, + producedTypeInfo, + rowDataConverter, + upsertMode); + } + + public void setProducedDataType(DataType producedDataType) { + this.producedDataType = producedDataType; + } + + public void setConnectorMetadataKeys(List metadataKeys) { + this.connectorMetadataKeys = metadataKeys; + } + + private int[] adjustValueProjectionByAppendConnectorMetadata( + int physicalValuePlusFormatMetadataArity) { + // Concat the Physical Fields (value only) with Format metadata projection. + final int[] physicalValuePlusFormatMetadataProjection = + IntStream.concat( + IntStream.of(valueProjection), + IntStream.range( + keyProjection.length + valueProjection.length, + physicalValuePlusFormatMetadataArity)) + .toArray(); + return physicalValuePlusFormatMetadataProjection; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableDeserializationSchemaFactory that = (PulsarTableDeserializationSchemaFactory) o; + return Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyDecodingFormat, that.keyDecodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Objects.equals(valueDecodingFormat, that.valueDecodingFormat) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(producedDataType, that.producedDataType) + && Objects.equals(connectorMetadataKeys, that.connectorMetadataKeys) + && Objects.equals(upsertMode, that.upsertMode); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + producedDataType, + connectorMetadataKeys, + upsertMode); + result = 31 * result + Arrays.hashCode(keyProjection); + result = 31 * result + Arrays.hashCode(valueProjection); + return result; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java new file mode 100644 index 0000000000000..92ee64b009ffd --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java @@ -0,0 +1,229 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +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.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +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 ScanTableSource} implementation for Pulsar SQL Connector. It uses a {@link + * SourceProvider} so it doesn't need to support {@link + * org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown} interface. + * + *

{@link PulsarTableSource} + */ +public class PulsarTableSource implements ScanTableSource, SupportsReadingMetadata { + // -------------------------------------------------------------------------------------------- + // Format attributes + // -------------------------------------------------------------------------------------------- + + private static final String FORMAT_METADATA_PREFIX = "value."; + + private final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory; + + /** + * Usually it is the same as the valueDecodingFormat, but use a different naming to show that it + * is used to list all the format metadata keys. + */ + private final DecodingFormat> decodingFormatForReadingMetadata; + + private final ChangelogMode changelogMode; + + // -------------------------------------------------------------------------------------------- + // PulsarSource needed attributes + // -------------------------------------------------------------------------------------------- + + private final List topics; + + private final Properties properties; + + private final StartCursor startCursor; + + private final StopCursor stopCursor; + + private final SubscriptionType subscriptionType; + + public PulsarTableSource( + PulsarTableDeserializationSchemaFactory deserializationSchemaFactory, + DecodingFormat> decodingFormatForReadingMetadata, + ChangelogMode changelogMode, + List topics, + Properties properties, + StartCursor startCursor, + StopCursor stopCursor, + SubscriptionType subscriptionType) { + // Format attributes + this.deserializationSchemaFactory = checkNotNull(deserializationSchemaFactory); + this.decodingFormatForReadingMetadata = checkNotNull(decodingFormatForReadingMetadata); + this.changelogMode = changelogMode; + // DataStream connector attributes + this.topics = topics; + this.properties = checkNotNull(properties); + this.startCursor = checkNotNull(startCursor); + this.stopCursor = checkNotNull(stopCursor); + this.subscriptionType = checkNotNull(subscriptionType); + } + + @Override + public ChangelogMode getChangelogMode() { + return changelogMode; + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { + PulsarDeserializationSchema deserializationSchema = + deserializationSchemaFactory.createPulsarDeserialization(context); + PulsarSource source = + PulsarSource.builder() + .setTopics(topics) + .setStartCursor(startCursor) + .setUnboundedStopCursor(stopCursor) + .setDeserializationSchema(deserializationSchema) + .setSubscriptionType(subscriptionType) + .setProperties(properties) + .build(); + return SourceProvider.of(source); + } + + /** + * According to convention, the order of the final row must be PHYSICAL + FORMAT METADATA + + * CONNECTOR METADATA where the format metadata has the highest precedence. + * + * @return + */ + @Override + public Map listReadableMetadata() { + final Map allMetadataMap = new LinkedHashMap<>(); + + // add value format metadata with prefix + decodingFormatForReadingMetadata + .listReadableMetadata() + .forEach((key, value) -> allMetadataMap.put(FORMAT_METADATA_PREFIX + key, value)); + // add connector metadata + Stream.of(PulsarReadableMetadata.ReadableMetadata.values()) + .forEachOrdered(m -> allMetadataMap.putIfAbsent(m.key, m.dataType)); + + return allMetadataMap; + } + + @Override + public void applyReadableMetadata(List allMetadataKeys, DataType producedDataType) { + // separate connector and format metadata + final List formatMetadataKeys = + allMetadataKeys.stream() + .filter(k -> k.startsWith(FORMAT_METADATA_PREFIX)) + .collect(Collectors.toList()); + + final List connectorMetadataKeys = new ArrayList<>(allMetadataKeys); + connectorMetadataKeys.removeAll(formatMetadataKeys); + + // push down format metadata + final Map formatMetadata = + decodingFormatForReadingMetadata.listReadableMetadata(); + if (formatMetadata.size() > 0) { + final List requestedFormatMetadataKeys = + formatMetadataKeys.stream() + .map(k -> k.substring(FORMAT_METADATA_PREFIX.length())) + .collect(Collectors.toList()); + decodingFormatForReadingMetadata.applyReadableMetadata(requestedFormatMetadataKeys); + } + + // update the factory attributes. + deserializationSchemaFactory.setConnectorMetadataKeys(connectorMetadataKeys); + deserializationSchemaFactory.setProducedDataType(producedDataType); + } + + @Override + public String asSummaryString() { + return "Pulsar table source"; + } + + @Override + public DynamicTableSource copy() { + final PulsarTableSource copy = + new PulsarTableSource( + deserializationSchemaFactory, + decodingFormatForReadingMetadata, + changelogMode, + topics, + properties, + startCursor, + stopCursor, + subscriptionType); + return copy; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSource that = (PulsarTableSource) o; + return Objects.equals(deserializationSchemaFactory, that.deserializationSchemaFactory) + && Objects.equals( + decodingFormatForReadingMetadata, that.decodingFormatForReadingMetadata) + && Objects.equals(changelogMode, that.changelogMode) + && Objects.equals(topics, that.topics) + && Objects.equals(properties, that.properties) + && Objects.equals(startCursor, that.startCursor) + && Objects.equals(stopCursor, that.stopCursor) + && subscriptionType == that.subscriptionType; + } + + @Override + public int hashCode() { + return Objects.hash( + deserializationSchemaFactory, + decodingFormatForReadingMetadata, + changelogMode, + topics, + properties, + startCursor, + stopCursor, + subscriptionType); + } +} From 7fe3a0ad0b2a3fbf0b01bf9007c951874582eeb4 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:24:48 +0800 Subject: [PATCH 02/20] b. Pulsar Table Sink implementation code --- .../sink/PulsarTableSerializationSchema.java | 131 ++++++++++++++ ...PulsarTableSerializationSchemaFactory.java | 160 ++++++++++++++++ .../pulsar/table/sink/PulsarTableSink.java | 171 ++++++++++++++++++ .../table/sink/PulsarWritableMetadata.java | 140 ++++++++++++++ 4 files changed, 602 insertions(+) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java new file mode 100644 index 0000000000000..68271438247b7 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -0,0 +1,131 @@ +/* + * 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 org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; + +import org.apache.pulsar.client.api.Schema; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link PulsarSerializationSchema} implementation for Pulsar SQL sink connector. It is + * responsible for retrieving fields from Flink row and serialize into Pulsar message key or body, + * and set necessary metadata fields as required. + */ +public class PulsarTableSerializationSchema implements PulsarSerializationSchema { + + private static final long serialVersionUID = 7314442107082067836L; + + @Nullable private final SerializationSchema keySerialization; + + private final RowData.FieldGetter[] keyFieldGetters; + + private final SerializationSchema valueSerialization; + + private final RowData.FieldGetter[] valueFieldGetters; + + private final PulsarWritableMetadata writableMetadata; + + private final boolean upsertMode; + + public PulsarTableSerializationSchema( + @Nullable SerializationSchema keySerialization, + RowData.FieldGetter[] keyFieldGetters, + SerializationSchema valueSerialization, + RowData.FieldGetter[] valueFieldGetters, + PulsarWritableMetadata writableMetadata, + boolean upsertMode) { + this.keySerialization = keySerialization; + this.keyFieldGetters = checkNotNull(keyFieldGetters); + this.valueSerialization = checkNotNull(valueSerialization); + this.valueFieldGetters = checkNotNull(valueFieldGetters); + this.writableMetadata = checkNotNull(writableMetadata); + this.upsertMode = upsertMode; + } + + @Override + public void open( + SerializationSchema.InitializationContext initializationContext, + PulsarSinkContext sinkContext, + SinkConfiguration sinkConfiguration) + throws Exception { + if (keySerialization != null) { + keySerialization.open(initializationContext); + } + valueSerialization.open(initializationContext); + } + + @Override + public PulsarMessage serialize(RowData consumedRow, PulsarSinkContext sinkContext) { + + PulsarMessageBuilder messageBuilder = new PulsarMessageBuilder<>(); + + final RowKind kind = consumedRow.getRowKind(); + final byte[] serializedData; + if (upsertMode) { + if (kind == RowKind.DELETE || kind == RowKind.UPDATE_BEFORE) { + // use null message as the tombstone message + serializedData = null; + } else { + final RowData valueRow = createProjectedRow(consumedRow, kind, valueFieldGetters); + valueRow.setRowKind(RowKind.INSERT); + serializedData = valueSerialization.serialize(valueRow); + } + } else { + final RowData valueRow = createProjectedRow(consumedRow, kind, valueFieldGetters); + serializedData = valueSerialization.serialize(valueRow); + } + + // apply metadata + writableMetadata.applyWritableMetadataInMessage(consumedRow, messageBuilder); + + // get key row data + if (keySerialization != null) { + final RowData keyRow = createProjectedRow(consumedRow, RowKind.INSERT, keyFieldGetters); + messageBuilder.keyBytes(keySerialization.serialize(keyRow)); + } + + messageBuilder.value(Schema.BYTES, serializedData); + return messageBuilder.build(); + } + + private static RowData createProjectedRow( + RowData consumedRow, RowKind kind, RowData.FieldGetter[] fieldGetters) { + final int arity = fieldGetters.length; + final GenericRowData genericRowData = new GenericRowData(kind, arity); + for (int fieldPos = 0; fieldPos < arity; fieldPos++) { + genericRowData.setField(fieldPos, fieldGetters[fieldPos].getFieldOrNull(consumedRow)); + } + return genericRowData; + } + + /** A class to read fields from Flink row and map to a Pulsar metadata. */ + public interface MetadataConverter extends Serializable { + Object read(RowData consumedRow, int pos); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java new file mode 100644 index 0000000000000..292c30b7e8a4d --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java @@ -0,0 +1,160 @@ +/* + * 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 org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.utils.DataTypeUtils; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A factory class which contains required fields mapping and encoding format information to + * construct a {@link PulsarTableSerializationSchema} instance. + */ +public class PulsarTableSerializationSchemaFactory { + + private final DataType physicalDataType; + + @Nullable private final EncodingFormat> keyEncodingFormat; + + private final int[] keyProjection; + + private final EncodingFormat> valueEncodingFormat; + + private final int[] valueProjection; + + /** Metadata that is appended at the end of a physical sink row. */ + private List writableMetadataKeys; + + private final boolean upsertMode; + + public PulsarTableSerializationSchemaFactory( + DataType physicalDataType, + @Nullable EncodingFormat> keyEncodingFormat, + int[] keyProjection, + EncodingFormat> valueEncodingFormat, + int[] valueProjection, + boolean upsertMode) { + this.physicalDataType = checkNotNull(physicalDataType); + this.keyEncodingFormat = keyEncodingFormat; + this.keyProjection = checkNotNull(keyProjection); + this.valueEncodingFormat = checkNotNull(valueEncodingFormat); + this.valueProjection = checkNotNull(valueProjection); + this.writableMetadataKeys = Collections.emptyList(); + this.upsertMode = upsertMode; + } + + public PulsarSerializationSchema createPulsarSerializationSchema( + DynamicTableSink.Context context) { + + final SerializationSchema keySerialization = + createSerialization(context, keyEncodingFormat, keyProjection, null); + + final SerializationSchema valueSerialization = + createSerialization(context, valueEncodingFormat, valueProjection, null); + final List physicalChildren = physicalDataType.getLogicalType().getChildren(); + + final RowData.FieldGetter[] keyFieldGetters = + getFieldGetters(physicalChildren, keyProjection); + final RowData.FieldGetter[] valueFieldGetters = + getFieldGetters(physicalChildren, valueProjection); + + final PulsarWritableMetadata writableMetadata = + new PulsarWritableMetadata(writableMetadataKeys, physicalChildren.size()); + + return new PulsarTableSerializationSchema( + keySerialization, + keyFieldGetters, + valueSerialization, + valueFieldGetters, + writableMetadata, + upsertMode); + } + + private @Nullable SerializationSchema createSerialization( + DynamicTableSink.Context context, + @Nullable EncodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeEncoder(context, physicalFormatDataType); + } + + private RowData.FieldGetter[] getFieldGetters( + List physicalChildren, int[] projection) { + return Arrays.stream(projection) + .mapToObj( + targetField -> + RowData.createFieldGetter( + physicalChildren.get(targetField), targetField)) + .toArray(RowData.FieldGetter[]::new); + } + + public void setWritableMetadataKeys(List writableMetadataKeys) { + this.writableMetadataKeys = writableMetadataKeys; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSerializationSchemaFactory that = (PulsarTableSerializationSchemaFactory) o; + return Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyEncodingFormat, that.keyEncodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Objects.equals(valueEncodingFormat, that.valueEncodingFormat) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(writableMetadataKeys, that.writableMetadataKeys) + && Objects.equals(upsertMode, that.upsertMode); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + writableMetadataKeys, + upsertMode); + result = 31 * result + Arrays.hashCode(keyProjection); + result = 31 * result + Arrays.hashCode(valueProjection); + return result; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java new file mode 100644 index 0000000000000..b290967873e5f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java @@ -0,0 +1,171 @@ +/* + * 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 org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.sink.PulsarSinkBuilder; +import org.apache.flink.connector.pulsar.sink.writer.delayer.FixedMessageDelayer; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Pulsar SQL Connector sink. It supports {@link SupportsWritingMetadata}. */ +public class PulsarTableSink implements DynamicTableSink, SupportsWritingMetadata { + + private final PulsarTableSerializationSchemaFactory serializationSchemaFactory; + + private final ChangelogMode changelogMode; + + private final List topics; + + private final Properties properties; + + private final DeliveryGuarantee deliveryGuarantee; + + @Nullable private final TopicRouter topicRouter; + + private final TopicRoutingMode topicRoutingMode; + + private final long messageDelayMillis; + + public PulsarTableSink( + PulsarTableSerializationSchemaFactory serializationSchemaFactory, + ChangelogMode changelogMode, + List topics, + Properties properties, + DeliveryGuarantee deliveryGuarantee, + @Nullable TopicRouter topicRouter, + TopicRoutingMode topicRoutingMode, + long messageDelayMillis) { + this.serializationSchemaFactory = checkNotNull(serializationSchemaFactory); + this.changelogMode = checkNotNull(changelogMode); + this.topics = checkNotNull(topics); + // Mutable attributes + this.properties = checkNotNull(properties); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + this.topicRouter = topicRouter; + this.topicRoutingMode = checkNotNull(topicRoutingMode); + this.messageDelayMillis = messageDelayMillis; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + return this.changelogMode; + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + + final PulsarSerializationSchema pulsarSerializationSchema = + serializationSchemaFactory.createPulsarSerializationSchema(context); + + final PulsarSinkBuilder pulsarSinkBuilder = + PulsarSink.builder() + .setSerializationSchema(pulsarSerializationSchema) + .setProperties(properties) + .setDeliveryGuarantee(deliveryGuarantee) + .setTopics(topics) + .setTopicRoutingMode(topicRoutingMode) + .delaySendingMessage(new FixedMessageDelayer<>(messageDelayMillis)); + + if (topicRouter != null) { + pulsarSinkBuilder.setTopicRouter(topicRouter); + } + return SinkV2Provider.of(pulsarSinkBuilder.build()); + } + + @Override + public String asSummaryString() { + return "Pulsar dynamic table sink"; + } + + @Override + public Map listWritableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(PulsarWritableMetadata.WritableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyWritableMetadata(List metadataKeys, DataType consumedDataType) { + this.serializationSchemaFactory.setWritableMetadataKeys(metadataKeys); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSink that = (PulsarTableSink) o; + return Objects.equals(serializationSchemaFactory, that.serializationSchemaFactory) + && Objects.equals(changelogMode, that.changelogMode) + && Objects.equals(topics, that.topics) + && Objects.equals(properties, that.properties) + && deliveryGuarantee == that.deliveryGuarantee + && Objects.equals(topicRouter, that.topicRouter) + && topicRoutingMode == that.topicRoutingMode + && messageDelayMillis == that.messageDelayMillis; + } + + @Override + public int hashCode() { + return Objects.hash( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + } + + @Override + public DynamicTableSink copy() { + final PulsarTableSink copy = + new PulsarTableSink( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + return copy; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java new file mode 100644 index 0000000000000..909b04095f925 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java @@ -0,0 +1,140 @@ +/* + * 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 org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import org.bouncycastle.util.Arrays; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Stream; + +/** A class used to manage metadata that is non virtual for Pulsar SQL sink connector. */ +public class PulsarWritableMetadata implements Serializable { + + private static final long serialVersionUID = 8117156158379846715L; + + private final List writableMetadataKeys; + + private final int physicalChildrenSize; + + /** + * Contains the position for each value of {@link WritableMetadata} in the consumed row or -1 if + * this metadata key is not used. + */ + private int[] metadataPositions; + + public PulsarWritableMetadata(List writableMetadataKeys, int physicalChildrenSize) { + this.writableMetadataKeys = writableMetadataKeys; + this.physicalChildrenSize = physicalChildrenSize; + this.metadataPositions = getMetadataPositions(); + } + + public void applyWritableMetadataInMessage( + RowData consumedRow, PulsarMessageBuilder messageBuilder) { + Map properties = readMetadata(consumedRow, WritableMetadata.PROPERTIES); + if (properties != null) { + messageBuilder.properties(properties); + } + final Long eventTime = readMetadata(consumedRow, WritableMetadata.EVENT_TIME); + if (eventTime != null && eventTime > 0) { + messageBuilder.eventTime(eventTime); + } + } + + @SuppressWarnings("unchecked") + private T readMetadata(RowData consumedRow, WritableMetadata metadata) { + if (Arrays.isNullOrEmpty(metadataPositions)) { + return null; + } + final int pos = metadataPositions[metadata.ordinal()]; + if (pos < 0) { + return null; + } + return (T) metadata.converter.read(consumedRow, pos); + } + + private int[] getMetadataPositions() { + return Stream.of(WritableMetadata.values()) + .mapToInt( + m -> { + final int pos = writableMetadataKeys.indexOf(m.key); + if (pos < 0) { + return -1; + } + return physicalChildrenSize + pos; + }) + .toArray(); + } + + /** A list of writable metadata used by Pulsar SQL sink connector. */ + public enum WritableMetadata { + PROPERTIES( + "properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .nullable(), + (row, pos) -> { + if (row.isNullAt(pos)) { + return null; + } + final MapData map = row.getMap(pos); + final ArrayData keyArray = map.keyArray(); + final ArrayData valueArray = map.valueArray(); + + final Properties properties = new Properties(); + for (int i = 0; i < keyArray.size(); i++) { + if (!keyArray.isNullAt(i) && !valueArray.isNullAt(i)) { + final String key = keyArray.getString(i).toString(); + final String value = valueArray.getString(i).toString(); + properties.put(key, value); + } + } + return properties; + }), + + EVENT_TIME( + "event_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + (row, pos) -> { + if (row.isNullAt(pos)) { + return null; + } + return row.getTimestamp(pos, 3).getMillisecond(); + }); + public final String key; + + public final DataType dataType; + + public final PulsarTableSerializationSchema.MetadataConverter converter; + + WritableMetadata( + String key, + DataType dataType, + PulsarTableSerializationSchema.MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} From 49b4fe580ca5111bc2ed43407e6fad625d40bd38 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:25:48 +0800 Subject: [PATCH 03/20] c. Pulsar Table Factory, config utils and validation logics --- .../pulsar/table/PulsarTableFactory.java | 305 ++++++++++++++++ .../pulsar/table/PulsarTableOptionUtils.java | 344 ++++++++++++++++++ .../pulsar/table/PulsarTableOptions.java | 282 ++++++++++++++ .../table/PulsarTableValidationUtils.java | 198 ++++++++++ 4 files changed, 1129 insertions(+) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java new file mode 100644 index 0000000000000..867025de457e4 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java @@ -0,0 +1,305 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.sink.PulsarSinkOptions; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.PulsarSourceOptions; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; +import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.source.PulsarTableSource; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createKeyFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createValueFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getKeyDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getKeyEncodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueEncodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.EXPLICIT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.VALUE_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validatePrimaryKeyConstraints; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSinkOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSourceOptions; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.apache.pulsar.shade.org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; + +/** + * Factory for creating {@link DynamicTableSource} and {@link DynamicTableSink}. + * + *

The main role of this class is to retrieve config options and validate options from config and + * the table schema. It also sets default values if a config option is not present. + */ +public class PulsarTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory { + + public static final String IDENTIFIER = "pulsar"; + + public static final String DEFAULT_SUBSCRIPTION_NAME_PREFIX = "flink-sql-connector-pulsar-"; + + public static final boolean UPSERT_DISABLED = false; + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + // Format options should be retrieved before validation. + final DecodingFormat> keyDecodingFormat = + getKeyDecodingFormat(helper); + final DecodingFormat> valueDecodingFormat = + getValueDecodingFormat(helper); + ReadableConfig tableOptions = helper.getOptions(); + + // Validate configs are not conflict; each options is consumed; no unwanted configs + // PulsarOptions, PulsarSourceOptions and PulsarSinkOptions is not part of the validation. + helper.validateExcept( + PulsarOptions.CLIENT_CONFIG_PREFIX, + PulsarOptions.ADMIN_CONFIG_PREFIX, + PulsarSourceOptions.SOURCE_CONFIG_PREFIX, + PulsarSourceOptions.CONSUMER_CONFIG_PREFIX, + PulsarSinkOptions.PRODUCER_CONFIG_PREFIX, + PulsarSinkOptions.SINK_CONFIG_PREFIX); + + validatePrimaryKeyConstraints( + context.getObjectIdentifier(), context.getPrimaryKeyIndexes(), helper); + + validateTableSourceOptions(tableOptions); + + // Retrieve configs + final List topics = getTopicListFromOptions(tableOptions); + final StartCursor startCursor = getStartCursor(tableOptions); + final StopCursor stopCursor = getStopCursor(tableOptions); + final SubscriptionType subscriptionType = getSubscriptionType(tableOptions); + + // Forward source configs + final Properties properties = getPulsarProperties(tableOptions); + properties.setProperty(PULSAR_ADMIN_URL.key(), tableOptions.get(ADMIN_URL)); + properties.setProperty(PULSAR_SERVICE_URL.key(), tableOptions.get(SERVICE_URL)); + // Set random subscriptionName if not provided + properties.setProperty( + PULSAR_SUBSCRIPTION_NAME.key(), + tableOptions + .getOptional(SOURCE_SUBSCRIPTION_NAME) + .orElse(DEFAULT_SUBSCRIPTION_NAME_PREFIX + randomAlphabetic(5))); + // Retrieve physical fields (not including computed or metadata fields), + // and projections and create a schema factory based on such information. + final DataType physicalDataType = context.getPhysicalRowDataType(); + + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + physicalDataType, + keyDecodingFormat, + keyProjection, + valueDecodingFormat, + valueProjection, + UPSERT_DISABLED); + + // Set default values for configuration not exposed to user. + final DecodingFormat> decodingFormatForMetadataPushdown = + valueDecodingFormat; + final ChangelogMode changelogMode = decodingFormatForMetadataPushdown.getChangelogMode(); + + return new PulsarTableSource( + deserializationSchemaFactory, + decodingFormatForMetadataPushdown, + changelogMode, + topics, + properties, + startCursor, + stopCursor, + subscriptionType); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + // Format options should be retrieved before validation. + final EncodingFormat> keyEncodingFormat = + getKeyEncodingFormat(helper); + final EncodingFormat> valueEncodingFormat = + getValueEncodingFormat(helper); + ReadableConfig tableOptions = helper.getOptions(); + + // Validate configs are not conflict; each options is consumed; no unwanted configs + // PulsarOptions, PulsarSourceOptions and PulsarSinkOptions is not part of the validation. + helper.validateExcept( + PulsarOptions.CLIENT_CONFIG_PREFIX, + PulsarOptions.ADMIN_CONFIG_PREFIX, + PulsarSourceOptions.SOURCE_CONFIG_PREFIX, + PulsarSourceOptions.CONSUMER_CONFIG_PREFIX, + PulsarSinkOptions.PRODUCER_CONFIG_PREFIX, + PulsarSinkOptions.SINK_CONFIG_PREFIX); + + validatePrimaryKeyConstraints( + context.getObjectIdentifier(), context.getPrimaryKeyIndexes(), helper); + + validateTableSinkOptions(tableOptions); + + // Retrieve configs + final TopicRouter topicRouter = + getTopicRouter(tableOptions, context.getClassLoader()); + final TopicRoutingMode topicRoutingMode = getTopicRoutingMode(tableOptions); + final long messageDelayMillis = getMessageDelayMillis(tableOptions); + + final List topics = getTopicListFromOptions(tableOptions); + + // Forward sink configs + final Properties properties = getPulsarProperties(tableOptions); + properties.setProperty(PULSAR_ADMIN_URL.key(), tableOptions.get(ADMIN_URL)); + properties.setProperty(PULSAR_SERVICE_URL.key(), tableOptions.get(SERVICE_URL)); + + // Retrieve physical DataType (not including computed or metadata fields) + final DataType physicalDataType = context.getPhysicalRowDataType(); + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + physicalDataType, + keyEncodingFormat, + keyProjection, + valueEncodingFormat, + valueProjection, + UPSERT_DISABLED); + + // Set default values for configuration not exposed to user. + final DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE; + final ChangelogMode changelogMode = valueEncodingFormat.getChangelogMode(); + + return new PulsarTableSink( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Stream.of(TOPICS, ADMIN_URL, SERVICE_URL).collect(Collectors.toSet()); + } + + @Override + public Set> optionalOptions() { + return Stream.of( + FactoryUtil.FORMAT, + VALUE_FORMAT, + SOURCE_SUBSCRIPTION_NAME, + SOURCE_SUBSCRIPTION_TYPE, + SOURCE_START_FROM_MESSAGE_ID, + SOURCE_START_FROM_PUBLISH_TIME, + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME, + SINK_CUSTOM_TOPIC_ROUTER, + SINK_TOPIC_ROUTING_MODE, + SINK_MESSAGE_DELAY_INTERVAL, + SINK_PARALLELISM, + KEY_FORMAT, + KEY_FIELDS, + EXPLICIT) + .collect(Collectors.toSet()); + } + + /** + * Format and Delivery guarantee related options are not forward options. + * + * @return + */ + @Override + public Set> forwardOptions() { + return Stream.of( + TOPICS, + ADMIN_URL, + SERVICE_URL, + SOURCE_SUBSCRIPTION_TYPE, + SOURCE_SUBSCRIPTION_NAME, + SOURCE_START_FROM_MESSAGE_ID, + SOURCE_START_FROM_PUBLISH_TIME, + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME, + SINK_CUSTOM_TOPIC_ROUTER, + SINK_TOPIC_ROUTING_MODE, + SINK_MESSAGE_DELAY_INTERVAL) + .collect(Collectors.toSet()); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java new file mode 100644 index 0000000000000..76f0f509d6b21 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java @@ -0,0 +1,344 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.VALUE_FORMAT; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A util class for getting fields from config options, getting formats and other useful + * information. + * + *

It contains the following functionalities. + * + *

+ */ +public class PulsarTableOptionUtils { + + private PulsarTableOptionUtils() {} + + public static final String TOPIC_LIST_DELIMITER = ";"; + + // -------------------------------------------------------------------------------------------- + // Decoding / Encoding and Projection + // -------------------------------------------------------------------------------------------- + + @Nullable + public static DecodingFormat> getKeyDecodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalDecodingFormat(DeserializationFormatFactory.class, KEY_FORMAT) + .orElse(null); + } + + @Nullable + public static EncodingFormat> getKeyEncodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT) + .orElse(null); + } + + public static DecodingFormat> getValueDecodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalDecodingFormat( + DeserializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> + helper.discoverDecodingFormat( + DeserializationFormatFactory.class, VALUE_FORMAT)); + } + + public static EncodingFormat> getValueEncodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalEncodingFormat( + SerializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> + helper.discoverEncodingFormat( + SerializationFormatFactory.class, VALUE_FORMAT)); + } + + /** + * Creates an array of indices that determine which physical fields of the table schema to + * include in the key format and the order that those fields have in the key format. + */ + public static int[] createKeyFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + checkArgument(physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + final Optional optionalKeyFormat = options.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = options.getOptional(KEY_FIELDS); + + if (!optionalKeyFormat.isPresent()) { + return new int[0]; + } + + final List keyFields = optionalKeyFields.get(); + final List physicalFields = LogicalTypeChecks.getFieldNames(physicalType); + return keyFields.stream() + .mapToInt( + keyField -> { + final int pos = physicalFields.indexOf(keyField); + // check that field name exists + if (pos < 0) { + throw new ValidationException( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option: %s", + keyField, KEY_FIELDS.key(), physicalFields)); + } + // check that field name is prefixed correctly + return pos; + }) + .toArray(); + } + + public static int[] createValueFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + checkArgument(physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + + final int physicalFieldCount = LogicalTypeChecks.getFieldCount(physicalType); + final IntStream physicalFields = IntStream.range(0, physicalFieldCount); + final int[] keyProjection = createKeyFormatProjection(options, physicalDataType); + return physicalFields + .filter(pos -> IntStream.of(keyProjection).noneMatch(k -> k == pos)) + .toArray(); + } + + // -------------------------------------------------------------------------------------------- + // Table Source Option Utils + // -------------------------------------------------------------------------------------------- + + public static List getTopicListFromOptions(ReadableConfig tableOptions) { + List topics = tableOptions.get(TOPICS); + return topics; + } + + public static Properties getPulsarProperties(ReadableConfig tableOptions) { + final Map configs = ((Configuration) tableOptions).toMap(); + return getPulsarProperties(configs); + } + + public static Properties getPulsarProperties(Map configs) { + return getPulsarPropertiesWithPrefix(configs, "pulsar"); + } + + public static Properties getPulsarPropertiesWithPrefix( + ReadableConfig tableOptions, String prefix) { + final Map configs = ((Configuration) tableOptions).toMap(); + return getPulsarPropertiesWithPrefix(configs, prefix); + } + + public static Properties getPulsarPropertiesWithPrefix( + Map configs, String prefix) { + final Properties pulsarProperties = new Properties(); + configs.keySet().stream() + .filter(key -> key.startsWith(prefix)) + .forEach(key -> pulsarProperties.put(key, configs.get(key))); + return pulsarProperties; + } + + public static StartCursor getStartCursor(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_START_FROM_MESSAGE_ID).isPresent()) { + return parseMessageIdStartCursor(tableOptions.get(SOURCE_START_FROM_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_START_FROM_PUBLISH_TIME).isPresent()) { + return parsePublishTimeStartCursor(tableOptions.get(SOURCE_START_FROM_PUBLISH_TIME)); + } else { + return StartCursor.earliest(); + } + } + + public static StopCursor getStopCursor(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_STOP_AT_MESSAGE_ID).isPresent()) { + return parseAtMessageIdStopCursor(tableOptions.get(SOURCE_STOP_AT_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_STOP_AFTER_MESSAGE_ID).isPresent()) { + return parseAfterMessageIdStopCursor(tableOptions.get(SOURCE_STOP_AFTER_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_STOP_AT_PUBLISH_TIME).isPresent()) { + return parseAtPublishTimeStopCursor(tableOptions.get(SOURCE_STOP_AT_PUBLISH_TIME)); + } else { + return StopCursor.never(); + } + } + + public static SubscriptionType getSubscriptionType(ReadableConfig tableOptions) { + return tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); + } + + protected static StartCursor parseMessageIdStartCursor(String config) { + if (Objects.equals(config, "earliest")) { + return StartCursor.earliest(); + } else if (Objects.equals(config, "latest")) { + return StartCursor.latest(); + } else { + return StartCursor.fromMessageId(parseMessageIdString(config)); + } + } + + protected static StartCursor parsePublishTimeStartCursor(Long config) { + return StartCursor.fromPublishTime(config); + } + + protected static StopCursor parseAtMessageIdStopCursor(String config) { + if (Objects.equals(config, "never")) { + return StopCursor.never(); + } else if (Objects.equals(config, "latest")) { + return StopCursor.latest(); + } else { + return StopCursor.atMessageId(parseMessageIdString(config)); + } + } + + protected static StopCursor parseAfterMessageIdStopCursor(String config) { + return StopCursor.afterMessageId(parseMessageIdString(config)); + } + + protected static StopCursor parseAtPublishTimeStopCursor(Long config) { + return StopCursor.atPublishTime(config); + } + + protected static MessageIdImpl parseMessageIdString(String config) { + String[] tokens = config.split(":", 3); + checkArgument(tokens.length == 3, "MessageId format must be ledgerId:entryId:partitionId."); + + try { + long ledgerId = Long.parseLong(tokens[0]); + long entryId = Long.parseLong(tokens[1]); + int partitionId = Integer.parseInt(tokens[2]); + MessageIdImpl messageId = new MessageIdImpl(ledgerId, entryId, partitionId); + return messageId; + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId. " + + "Each id should be able to parsed to long type."); + } + } + + // -------------------------------------------------------------------------------------------- + // Table Sink Option Utils + // -------------------------------------------------------------------------------------------- + + public static TopicRouter getTopicRouter( + ReadableConfig readableConfig, ClassLoader classLoader) { + if (!readableConfig.getOptional(SINK_CUSTOM_TOPIC_ROUTER).isPresent()) { + return null; + } + + String className = readableConfig.get(SINK_CUSTOM_TOPIC_ROUTER); + try { + Class clazz = Class.forName(className, true, classLoader); + if (!TopicRouter.class.isAssignableFrom(clazz)) { + throw new ValidationException( + String.format( + "Sink TopicRouter class '%s' should extend from the required class %s", + className, TopicRouter.class.getName())); + } + @SuppressWarnings("unchecked") + final TopicRouter topicRouter = + InstantiationUtil.instantiate(className, TopicRouter.class, classLoader); + + return topicRouter; + } catch (ClassNotFoundException | FlinkException e) { + throw new ValidationException( + String.format( + "Could not find and instantiate TopicRouter class '%s'", className), + e); + } + } + + public static TopicRoutingMode getTopicRoutingMode(ReadableConfig readableConfig) { + return readableConfig.get(SINK_TOPIC_ROUTING_MODE); + } + + public static long getMessageDelayMillis(ReadableConfig readableConfig) { + return readableConfig.get(SINK_MESSAGE_DELAY_INTERVAL).toMillis(); + } + + // -------------------------------------------------------------------------------------------- + // Table Topic Name Utils + // -------------------------------------------------------------------------------------------- + + public static boolean hasMultipleTopics(String topicsConfigString) { + checkNotNull(topicsConfigString); + String[] topics = topicsConfigString.split(TOPIC_LIST_DELIMITER); + return topics.length > 1; + } + + public static String getFirstTopic(String topicsConfigString) { + checkNotNull(topicsConfigString); + String[] topics = topicsConfigString.split(TOPIC_LIST_DELIMITER); + checkArgument(topics.length > 0); + return topics[0]; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java new file mode 100644 index 0000000000000..2f3d9d602a3db --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java @@ -0,0 +1,282 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.code; +import static org.apache.flink.configuration.description.TextElement.text; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT_SUFFIX; + +/** + * Config options that is used to configure a Pulsar SQL Connector. These config options are + * specific to SQL Connectors only. Other runtime configurations can be found in {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions}, {@link + * org.apache.flink.connector.pulsar.source.PulsarSourceOptions}, and {@link + * org.apache.flink.connector.pulsar.sink.PulsarSinkOptions}. + */ +@PublicEvolving +public final class PulsarTableOptions { + + private PulsarTableOptions() {} + + public static final ConfigOption> TOPICS = + ConfigOptions.key("topics") + .stringType() + .asList() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (%s) like %s.", + code(";"), code("topic-1;topic-2")) + .build()); + + // -------------------------------------------------------------------------------------------- + // Table Source Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SOURCE_SUBSCRIPTION_TYPE = + ConfigOptions.key("source.subscription-type") + .enumType(SubscriptionType.class) + .defaultValue(SubscriptionType.Exclusive) + .withDescription( + Description.builder() + .text( + "The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only %s and %s subscription types are supported.", + code("Exclusive"), code("Shared")) + .build()); + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.source.PulsarSourceOptions#PULSAR_SUBSCRIPTION_NAME}. + * Copied because we want to have a default value for it. + */ + public static final ConfigOption SOURCE_SUBSCRIPTION_NAME = + ConfigOptions.key("source.subscription-name") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer.") + .build()); + + public static final ConfigOption SOURCE_START_FROM_MESSAGE_ID = + ConfigOptions.key("source.start.message-id") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Optional message id used to specify a consuming starting point for " + + "source. Use %s, %s or pass in a message id " + + "representation in %s, " + + "such as %s", + code("earliest"), + code("latest"), + code("ledgerId:entryId:partitionId"), + code("12:2:-1")) + .build()); + + public static final ConfigOption SOURCE_START_FROM_PUBLISH_TIME = + ConfigOptions.key("source.start.publish-time") + .longType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "(Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data.") + .build()); + + public static final ConfigOption SOURCE_STOP_AT_MESSAGE_ID = + ConfigOptions.key("source.stop.at-message-id") + .stringType() + .noDefaultValue() + .withDescription( + "Optional message id used to specify a stop cursor for the unbounded sql " + + "source. Use \"never\", \"latest\" or pass in a message id " + + "representation in \"ledgerId:entryId:partitionId\", " + + "such as \"12:2:-1\""); + + public static final ConfigOption SOURCE_STOP_AFTER_MESSAGE_ID = + ConfigOptions.key("source.stop.after-message-id") + .stringType() + .noDefaultValue() + .withDescription( + "Optional message id used to specify a stop position but include the " + + "given message in the consuming result for the unbounded sql " + + "source. Pass in a message id " + + "representation in \"ledgerId:entryId:partitionId\", " + + "such as \"12:2:-1\". "); + + public static final ConfigOption SOURCE_STOP_AT_PUBLISH_TIME = + ConfigOptions.key("source.stop.at-publish-time") + .longType() + .noDefaultValue() + .withDescription( + "Optional publish timestamp used to specify a stop cursor" + + " for the unbounded sql source."); + + // -------------------------------------------------------------------------------------------- + // Table Sink Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SINK_CUSTOM_TOPIC_ROUTER = + ConfigOptions.key("sink.custom-topic-router") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "(Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the %s option will be ignored.", + code("sink.topic-routing-mode")) + .build()); + + public static final ConfigOption SINK_TOPIC_ROUTING_MODE = + ConfigOptions.key("sink.topic-routing-mode") + .enumType(TopicRoutingMode.class) + .defaultValue(TopicRoutingMode.ROUND_ROBIN) + .withDescription( + Description.builder() + .text( + "(Optional) the topic routing mode. Available options are %s and %s. By default, it is set to %s. If you want to use a custom topic router, use the %s option to determine the partition for a particular message.", + code("round-robin"), + code("message-key-hash"), + code("round-robin"), + code("sink.custom-topic-router")) + .build()); + + public static final ConfigOption SINK_MESSAGE_DELAY_INTERVAL = + ConfigOptions.key("sink.message-delay-interval") + .durationType() + .defaultValue(Duration.ZERO) + .withDescription( + "(Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink)."); + + // -------------------------------------------------------------------------------------------- + // Format Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption KEY_FORMAT = + ConfigOptions.key("key" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for decoding/encoding key bytes in " + + "Pulsar message. The identifier is used to discover a suitable format factory."); + + public static final ConfigOption> KEY_FIELDS = + ConfigOptions.key("key.fields") + .stringType() + .asList() + .defaultValues() + .withDescription( + "An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined."); + + public static final ConfigOption VALUE_FORMAT = + ConfigOptions.key("value" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for decoding/encoding value data. " + + "The identifier is used to discover a suitable format factory."); + + // -------------------------------------------------------------------------------------------- + // Pulsar Options + // -------------------------------------------------------------------------------------------- + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions#PULSAR_ADMIN_URL}. Copied here + * because it is a required config option and should not be included in the {@link + * org.apache.flink.table.factories.FactoryUtil.FactoryHelper#validateExcept(String...)} method. + * + *

By default all {@link org.apache.flink.connector.pulsar.common.config.PulsarOptions} are + * included in the validateExcept() method./p> + */ + public static final ConfigOption ADMIN_URL = + ConfigOptions.key("admin-url") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "The Pulsar service HTTP URL for the admin endpoint. For example, %s, or %s for TLS.", + code("http://my-broker.example.com:8080"), + code("https://my-broker.example.com:8443")) + .build()); + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions#PULSAR_SERVICE_URL}. Copied + * here because it is a required config option and should not be included in the {@link + * org.apache.flink.table.factories.FactoryUtil.FactoryHelper#validateExcept(String...)} method. + * + *

By default all {@link org.apache.flink.connector.pulsar.common.config.PulsarOptions} are + * included in the validateExcept() method./p> + */ + public static final ConfigOption SERVICE_URL = + ConfigOptions.key("service-url") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("Service URL provider for Pulsar service.") + .linebreak() + .text( + "To connect to Pulsar using client libraries, you need to specify a Pulsar protocol URL.") + .linebreak() + .text( + "You can assign Pulsar protocol URLs to specific clusters and use the Pulsar scheme.") + .linebreak() + .list( + text( + "This is an example of %s: %s.", + code("localhost"), + code("pulsar://localhost:6650")), + text( + "If you have multiple brokers, the URL is as: %s", + code( + "pulsar://localhost:6550,localhost:6651,localhost:6652")), + text( + "A URL for a production Pulsar cluster is as: %s", + code( + "pulsar://pulsar.us-west.example.com:6650")), + text( + "If you use TLS authentication, the URL is as %s", + code( + "pulsar+ssl://pulsar.us-west.example.com:6651"))) + .build()); + + public static final ConfigOption EXPLICIT = + ConfigOptions.key("explicit") + .booleanType() + .defaultValue(true) + .withDescription("Indicate if the table is an explicit Flink table."); +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java new file mode 100644 index 0000000000000..42f8855d5082c --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java @@ -0,0 +1,198 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.types.RowKind; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.pulsar.common.naming.TopicName.isValid; + +/** Util class for source and sink validation rules. */ +public class PulsarTableValidationUtils { + + private PulsarTableValidationUtils() {} + + public static void validatePrimaryKeyConstraints( + ObjectIdentifier tableName, + int[] primaryKeyIndexes, + FactoryUtil.TableFactoryHelper helper) { + final DecodingFormat> format = + getValueDecodingFormat(helper); + if (primaryKeyIndexes.length > 0 + && format.getChangelogMode().containsOnly(RowKind.INSERT)) { + throw new ValidationException( + String.format( + "The Pulsar table '%s' with '%s' format doesn't support defining PRIMARY KEY constraint" + + " on the table, because it can't guarantee the semantic of primary key.", + tableName.asSummaryString(), format)); + } + } + + public static void validateTableSourceOptions(ReadableConfig tableOptions) { + validateTopicsConfigs(tableOptions); + validateStartCursorConfigs(tableOptions); + validateStopCursorConfigs(tableOptions); + validateSubscriptionTypeConfigs(tableOptions); + validateKeyFormatConfigs(tableOptions); + } + + public static void validateTableSinkOptions(ReadableConfig tableOptions) { + validateTopicsConfigs(tableOptions); + validateKeyFormatConfigs(tableOptions); + validateSinkRoutingConfigs(tableOptions); + } + + protected static void validateTopicsConfigs(ReadableConfig tableOptions) { + if (tableOptions.get(TOPICS).isEmpty()) { + throw new ValidationException("The topics list should not be empty."); + } + + for (String topic : tableOptions.get(TOPICS)) { + if (!isValid(topic)) { + throw new ValidationException( + String.format("The topics name %s is not a valid topic name.", topic)); + } + } + } + + protected static void validateStartCursorConfigs(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_START_FROM_MESSAGE_ID).isPresent() + && tableOptions.getOptional(SOURCE_START_FROM_PUBLISH_TIME).isPresent()) { + throw new ValidationException( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME)); + } + } + + protected static void validateStopCursorConfigs(ReadableConfig tableOptions) { + Set> conflictConfigOptions = + Sets.newHashSet( + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME); + + long configsNums = + conflictConfigOptions.stream() + .map(tableOptions::getOptional) + .filter(Optional::isPresent) + .count(); + + if (configsNums > 1) { + throw new ValidationException( + String.format( + "Only one of %s, %s and %s can be specified. Detected more than 1 of them", + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME)); + } + } + + protected static void validateSubscriptionTypeConfigs(ReadableConfig tableOptions) { + SubscriptionType subscriptionType = tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); + + if (subscriptionType == SubscriptionType.Failover) { + throw new ValidationException( + String.format( + "%s SubscriptionType is not supported. ", SubscriptionType.Failover)); + } + } + + protected static void validateKeyFormatConfigs(ReadableConfig tableOptions) { + final Optional optionalKeyFormat = tableOptions.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = tableOptions.getOptional(KEY_FIELDS); + if (!optionalKeyFormat.isPresent() && optionalKeyFields.isPresent()) { + throw new ValidationException( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + } else if (optionalKeyFormat.isPresent() + && (!optionalKeyFields.isPresent() || optionalKeyFields.get().size() == 0)) { + throw new ValidationException( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + } + } + + protected static void validateSinkRoutingConfigs(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SINK_TOPIC_ROUTING_MODE).orElse(TopicRoutingMode.ROUND_ROBIN) + == TopicRoutingMode.CUSTOM) { + throw new ValidationException( + String.format( + "Only %s and %s can be used. For %s, please use sink.custom-topic-router for" + + "custom topic router and do not set this config.", + TopicRoutingMode.ROUND_ROBIN, + TopicRoutingMode.MESSAGE_KEY_HASH, + TopicRoutingMode.CUSTOM)); + } + if (tableOptions.getOptional(SINK_CUSTOM_TOPIC_ROUTER).isPresent() + && tableOptions.getOptional(SINK_TOPIC_ROUTING_MODE).isPresent()) { + throw new ValidationException( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE)); + } + } + + protected static void validateUpsertModeKeyConstraints( + ReadableConfig tableOptions, int[] primaryKeyIndexes) { + if (!tableOptions.getOptional(KEY_FIELDS).isPresent()) { + throw new ValidationException( + String.format( + "Upsert mode requires key.fields set to the primary key fields, should be set")); + } + + if (tableOptions.getOptional(KEY_FIELDS).get().size() == 0 + || primaryKeyIndexes.length == 0) { + throw new ValidationException( + "'upsert-pulsar' require to define a PRIMARY KEY constraint. " + + "The PRIMARY KEY specifies which columns should be read from or write to the Pulsar message key. " + + "The PRIMARY KEY also defines records in the 'upsert-pulsar' table should update or delete on which keys."); + } + } +} From c5f1e6144488af94ed44ade4eb447c98433444de Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:28:31 +0800 Subject: [PATCH 04/20] d. test utils classes: mocks, POJO and static util classes --- .../testutils/MockPulsarAuthentication.java | 84 +++++++++++++++++++ .../table/testutils/MockTopicRouter.java | 37 ++++++++ .../table/testutils/PulsarTableTestUtils.java | 52 ++++++++++++ .../pulsar/table/testutils/TestingUser.java | 57 +++++++++++++ 4 files changed, 230 insertions(+) create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java new file mode 100644 index 0000000000000..0309d1275a20d --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java @@ -0,0 +1,84 @@ +/* + * 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.connector.pulsar.table.testutils; + +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.auth.AuthenticationDataNull; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +/** A class to verify Pulsar authentication auth params map is created properly. */ +public class MockPulsarAuthentication implements Authentication { + public static String key1 = "key1"; + public static String key2 = "key2"; + public static String value1 = "value1"; + public static String value2 = "value2"; + + @Override + public String getAuthMethodName() { + return "custom authentication"; + } + + @Override + public AuthenticationDataProvider getAuthData() { + return new AuthenticationDataNull(); + } + + @Override + public AuthenticationDataProvider getAuthData(String brokerHostName) { + return new AuthenticationDataNull(); + } + + @Override + public void authenticationStage( + String requestUrl, + AuthenticationDataProvider authData, + Map previousResHeaders, + CompletableFuture> authFuture) { + Authentication.super.authenticationStage( + requestUrl, authData, previousResHeaders, authFuture); + } + + @Override + public Set> newRequestHeader( + String hostName, + AuthenticationDataProvider authData, + Map previousResHeaders) { + return new HashSet<>(); + } + + @Override + public void configure(Map authParams) { + assert Objects.equals(authParams.get(key1), value1); + assert Objects.equals(authParams.get(key2), value2); + } + + @Override + public void start() throws PulsarClientException {} + + @Override + public void close() throws IOException {} +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java new file mode 100644 index 0000000000000..b5ef53841aaee --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java @@ -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.connector.pulsar.table.testutils; + +import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.table.data.RowData; + +import java.util.List; + +/** A mock topic Router for testing purposes only. */ +public class MockTopicRouter implements TopicRouter { + + private static final long serialVersionUID = 1316133122715449818L; + + @Override + public String route( + RowData rowData, String key, List partitions, PulsarSinkContext context) { + return "never-exist-topic"; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java new file mode 100644 index 0000000000000..f43be2d63eebf --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java @@ -0,0 +1,52 @@ +/* + * 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.connector.pulsar.table.testutils; + +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** Util class for verify testing results. */ +public class PulsarTableTestUtils { + public static List collectRows(Table table, int expectedSize) throws Exception { + final TableResult result = table.execute(); + final List collectedRows = new ArrayList<>(); + try (CloseableIterator iterator = result.collect()) { + while (collectedRows.size() < expectedSize && iterator.hasNext()) { + collectedRows.add(iterator.next()); + } + } + result.getJobClient() + .ifPresent( + jc -> { + try { + jc.cancel().get(5, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + return collectedRows; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java new file mode 100644 index 0000000000000..1cedc2e5b9766 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java @@ -0,0 +1,57 @@ +package org.apache.flink.connector.pulsar.table.testutils; + +import java.io.Serializable; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; + +/** + * A test POJO class used by table integration tests to validate the JSON and AVRO schema are + * compatible with corresponding Flink formats + */ +public class TestingUser implements Serializable { + private static final long serialVersionUID = -1123545861004770003L; + public String name; + public Integer age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestingUser that = (TestingUser) o; + return Objects.equals(name, that.name) && Objects.equals(age, that.age); + } + + @Override + public int hashCode() { + return Objects.hash(name, age); + } + + public static TestingUser createRandomUser() { + TestingUser user = new TestingUser(); + user.setName(randomAlphabetic(5)); + user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); + return user; + } +} From 1cbb48832f908509784e70303a0cb6cf5ec841f2 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:29:30 +0800 Subject: [PATCH 05/20] e. PulsarTableTestBase and IT Case --- .../pulsar/table/PulsarTableITCase.java | 620 ++++++++++++++++++ .../pulsar/table/PulsarTableTestBase.java | 84 +++ 2 files changed, 704 insertions(+) create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java new file mode 100644 index 0000000000000..4eada34491be6 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -0,0 +1,620 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.connector.pulsar.table.testutils.TestingUser; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.SuccessException; +import org.apache.flink.types.Row; + +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow; +import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; +import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createRandomUser; +import static org.apache.flink.util.CollectionUtil.entry; +import static org.apache.flink.util.CollectionUtil.map; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** + * IT cases for the Pulsar table source and sink. It aims to verify runtime behaviour and certain + * use cases are correct and can produce/consume the desired records as user specifies. + */ +@ExtendWith(MiniClusterExtension.class) +public class PulsarTableITCase extends PulsarTableTestBase { + + protected static final String JSON_FORMAT = "json"; + protected static final String AVRO_FORMAT = "avro"; + protected static final String CSV_FORMAT = "csv"; + protected static final String RAW_FORMAT = "raw"; + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT}) + void pulsarSourceSink(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "test_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + // ---------- Produce an event time stream into Pulsar ------------------- + String randomTableName = randomAlphabetic(5); + final String createTable = + String.format( + "create table %s (\n" + + " `computed-price` as price + 1.0,\n" + + " price decimal(38, 18),\n" + + " currency string,\n" + + " log_date date,\n" + + " log_time time(3),\n" + + " log_ts timestamp(3),\n" + + " ts as log_ts + INTERVAL '1' SECOND,\n" + + " watermark for ts as ts\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + randomTableName, + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createTable); + + String initialValues = + String.format( + "INSERT INTO %s\n" + + "SELECT CAST(price AS DECIMAL(10, 2)), currency, " + + " CAST(d AS DATE), CAST(t AS TIME(0)), CAST(ts AS TIMESTAMP(3))\n" + + "FROM (VALUES (2.02,'Euro','2019-12-12', '00:00:01', '2019-12-12 00:00:01.001001'), \n" + + " (1.11,'US Dollar','2019-12-12', '00:00:02', '2019-12-12 00:00:02.002001'), \n" + + " (50,'Yen','2019-12-12', '00:00:03', '2019-12-12 00:00:03.004001'), \n" + + " (3.1,'Euro','2019-12-12', '00:00:04', '2019-12-12 00:00:04.005001'), \n" + + " (5.33,'US Dollar','2019-12-12', '00:00:05', '2019-12-12 00:00:05.006001'), \n" + + " (0,'DUMMY','2019-12-12', '00:00:10', '2019-12-12 00:00:10'))\n" + + " AS orders (price, currency, d, t, ts)", + randomTableName); + tableEnv.executeSql(initialValues).await(); + + String query = + String.format( + "SELECT\n" + + " CAST(TUMBLE_END(ts, INTERVAL '5' SECOND) AS VARCHAR),\n" + + " CAST(MAX(log_date) AS VARCHAR),\n" + + " CAST(MAX(log_time) AS VARCHAR),\n" + + " CAST(MAX(ts) AS VARCHAR),\n" + + " COUNT(*),\n" + + " CAST(MAX(price) AS DECIMAL(10, 2))\n" + + "FROM %s\n" + + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)", + randomTableName); + + DataStream result = tableEnv.toDataStream(tableEnv.sqlQuery(query)); + TestingSinkFunction sink = new TestingSinkFunction(2); + result.addSink(sink).setParallelism(1); + + try { + env.execute("Job_2"); + } catch (Throwable e) { + if (!isCausedByJobFinished(e)) { + // re-throw + throw e; + } + } + + List expected = + Arrays.asList( + "+I[2019-12-12 00:00:05.000, 2019-12-12, 00:00:03, 2019-12-12 00:00:04.004, 3, 50.00]", + "+I[2019-12-12 00:00:10.000, 2019-12-12, 00:00:05, 2019-12-12 00:00:06.006, 2, 5.33]"); + + assertThat(TestingSinkFunction.rows).isEqualTo(expected); + } + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT}) + void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "key_partial_value_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + // ---------- Produce an event time stream into Pulsar ------------------- + + // k_user_id and user_id have different data types to verify the correct mapping, + // fields are reordered on purpose + String randomTableName = randomAlphabetic(5); + final String createTable = + String.format( + "CREATE TABLE %s (\n" + + " `user_id` BIGINT,\n" + + " `name` STRING,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'key.fields' = 'user_id; event_id'\n" + + ")", + randomTableName, + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format, + format); + + tableEnv.executeSql(createTable); + + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " (1, 'name 1', 100, 'payload 1'),\n" + + " (2, 'name 2', 101, 'payload 2'),\n" + + " (3, 'name 3', 102, 'payload 3')", + randomTableName); + tableEnv.executeSql(initialValues).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", randomTableName)), 3); + + assertThat(result) + .containsExactlyInAnyOrder( + Row.of(1L, "name 1", 100L, "payload 1"), + Row.of(2L, "name 2", 101L, "payload 2"), + Row.of(3L, "name 3", 102L, "payload 3")); + } + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT}) + void pulsarSourceSinkWithMetadata(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "metadata_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + String randomTableName = randomAlphabetic(5); + final String createTable = + String.format( + "CREATE TABLE %s (\n" + + " `physical_1` STRING,\n" + + " `physical_2` INT,\n" + + " `event_time` TIMESTAMP(3) METADATA,\n" + + " `properties` MAP METADATA,\n" + + " `physical_3` BOOLEAN\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'pulsar.producer.producerName' = 'pulsar-table-test',\n" + + " 'format' = '%s'\n" + + ")", + randomTableName, + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + tableEnv.executeSql(createTable); + + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TIMESTAMP '2022-03-24 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE),\n" + + " ('data 2', 2, TIMESTAMP '2022-03-25 13:12:11.123', CAST(NULL AS MAP), FALSE),\n" + + " ('data 3', 3, TIMESTAMP '2022-03-26 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE)", + randomTableName); + tableEnv.executeSql(initialValues).await(); + + // ---------- Consume stream from Pulsar ------------------- + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", randomTableName)), 3); + assertThat(result) + .containsExactlyInAnyOrder( + Row.of( + "data 1", + 1, + LocalDateTime.parse("2022-03-24T13:12:11.123"), + map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), + true), + Row.of( + "data 2", + 2, + LocalDateTime.parse("2022-03-25T13:12:11.123"), + Collections.emptyMap(), + false), + Row.of( + "data 3", + 3, + LocalDateTime.parse("2022-03-26T13:12:11.123"), + map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), + true)); + } + + @Test + void sendMessageWithPropertiesAndReadPropertiesMetadata() throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + + // create producer and send one message + String value = randomAlphabetic(5); + Map properties = new HashMap<>(); + properties.put("key1", "value1"); + properties.put("key2", "value2"); + try (Producer producer = + pulsar.operator().createProducer(sourceTopic, Schema.STRING)) { + producer.newMessage().value(value).properties(properties).send(); + } catch (PulsarClientException e) { + sneakyThrow(e); + } + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n," + + " `properties` MAP METADATA\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result) + .containsExactlyInAnyOrder( + Row.of(value, map(entry("key1", "value1"), entry("key2", "value2")))); + } + + @ParameterizedTest + @MethodSource("provideSchemaData") + void readAndSelectIntoTableUsingSimpleSchema( + String format, Schema schema, T value, String flinkTableDataType) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, schema, value); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " `field_1` %s\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + flinkTableDataType, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result).containsExactlyInAnyOrder(Row.of(value)); + + // insert into ... select from + + final String sinkTopic = "sink_topic_" + randomAlphanumeric(3); + createTestTopic(sinkTopic, 1); + + String sinkTableName = randomAlphabetic(5); + pulsar.operator().sendMessage(sourceTopic, schema, value); + final String createSinkTable = + String.format( + "create table %s (\n" + + " `field_1` %s\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sinkTableName, + flinkTableDataType, + PulsarTableFactory.IDENTIFIER, + sinkTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSinkTable); + tableEnv.executeSql( + String.format("INSERT INTO %s SELECT * FROM %s", sinkTableName, sourceTableName)); + Message sinkResult = pulsar.operator().receiveMessage(sinkTopic, schema); + assertThat(sinkResult.getValue()).isEqualTo(value); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void sendMessageToTopicAndReadUsingAvroBasedSchema( + String format, Schema schema, TestingUser value) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, schema, value); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " age INT,\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result).containsExactlyInAnyOrder(Row.of(value.getAge(), value.getName())); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void writeAndReadUsingAvroBasedSchema( + String format, Schema schema, TestingUser value) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING,\n" + + " age INT\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + String initialValues = + String.format( + "INSERT INTO %s\n" + "VALUES\n" + " ('%s', %s)", + sourceTableName, value.getName(), value.getAge()); + tableEnv.executeSql(initialValues).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result).containsExactlyInAnyOrder(Row.of(value.getName(), value.getAge())); + } + + @Test + void sendMessageToTopicAndReadUntilBoundedStopCursor() { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, Schema.STRING, randomAlphabetic(5)); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'pulsar.source.partitionDiscoveryIntervalMs' = '-1',\n" + + " 'source.stop.at-message-id' = 'latest',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + JobClient jobClient = + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)) + .execute() + .getJobClient() + .get(); + assertThatNoException() + .isThrownBy( + () -> { + JobExecutionResult result = + jobClient.getJobExecutionResult().get(1, TimeUnit.MINUTES); + }); + } + + @Test + void sendMessageToTopicAndReadUntilBoundedStopCursorButHasPartitionDiscovery() { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, Schema.STRING, randomAlphabetic(5)); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'source.stop.at-message-id' = 'latest',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + JobClient jobClient = + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)) + .execute() + .getJobClient() + .get(); + assertThatExceptionOfType(TimeoutException.class) + .isThrownBy( + () -> { + JobExecutionResult result = + jobClient.getJobExecutionResult().get(1, TimeUnit.MINUTES); + }); + } + + private static final class TestingSinkFunction implements SinkFunction { + + private static final long serialVersionUID = 455430015321124493L; + private static List rows = new ArrayList<>(); + + private final int expectedSize; + + private TestingSinkFunction(int expectedSize) { + this.expectedSize = expectedSize; + rows.clear(); + } + + @Override + public void invoke(Row value, Context context) { + rows.add(value.toString()); + if (rows.size() >= expectedSize) { + // job finish + throw new SuccessException(); + } + } + } + + private static boolean isCausedByJobFinished(Throwable e) { + if (e instanceof SuccessException) { + return true; + } else if (e.getCause() != null) { + return isCausedByJobFinished(e.getCause()); + } else { + return false; + } + } + + private static Stream provideSchemaData() { + return Stream.of( + Arguments.of(RAW_FORMAT, Schema.INT8, (byte) 0xa, DataTypes.TINYINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT16, Short.MAX_VALUE, DataTypes.SMALLINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT32, Integer.MAX_VALUE, DataTypes.INT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT64, Long.MAX_VALUE, DataTypes.BIGINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.FLOAT, Float.MAX_VALUE, DataTypes.FLOAT().toString()), + Arguments.of( + RAW_FORMAT, Schema.DOUBLE, Double.MAX_VALUE, DataTypes.DOUBLE().toString()), + Arguments.of(RAW_FORMAT, Schema.BOOL, Boolean.TRUE, DataTypes.BOOLEAN().toString()), + Arguments.of(RAW_FORMAT, Schema.BYTES, new byte[1], DataTypes.BYTES().toString()), + Arguments.of( + RAW_FORMAT, + Schema.STRING, + "this is a string", + DataTypes.STRING().toString())); + } + + private static Stream provideAvroBasedSchemaData() { + return Stream.of( + Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class), createRandomUser()), + Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class), createRandomUser())); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java new file mode 100644 index 0000000000000..24a4e7b9c6e1f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -0,0 +1,84 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Base class for Pulsar table integration test. */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class PulsarTableTestBase { + private static final Logger LOG = LoggerFactory.getLogger(PulsarTableTestBase.class); + + @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + // Defines pulsar running environment + @TestExternalSystem + protected PulsarTestEnvironment pulsar = new PulsarTestEnvironment(runtime()); + + @TestSemantics + protected CheckpointingMode[] semantics = + new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + + protected StreamExecutionEnvironment env; + + protected StreamTableEnvironment tableEnv; + + protected PulsarRuntime runtime() { + return PulsarRuntime.container(); + } + + private static final int DEFAULT_PARALLELISM = 1; + + @BeforeAll + public void beforeAll() { + pulsar.startUp(); + // run env + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(DEFAULT_PARALLELISM); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + tableEnv = StreamTableEnvironment.create(env); + tableEnv.getConfig() + .getConfiguration() + .setString("table.dynamic-table-options.enabled", "true"); + } + + public void createTestTopic(String topic, int numPartitions) { + pulsar.operator().createTopic(topic, numPartitions); + } + + @AfterAll + public void afterAll() { + pulsar.tearDown(); + } +} From 4e8f43ff1267656375bc6622ba7a2b9055c5e9fe Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:29:57 +0800 Subject: [PATCH 06/20] f. PulsarTableFactoryTest --- .../pulsar/table/PulsarTableFactoryTest.java | 441 ++++++++++++++++++ 1 file changed, 441 insertions(+) create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java new file mode 100644 index 0000000000000..361fa9d00daf6 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java @@ -0,0 +1,441 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.table.PulsarTableFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; +import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.source.PulsarTableSource; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.WatermarkSpec; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.utils.ResolvedExpressionMock; +import org.apache.flink.table.factories.TestFormatFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.assertj.core.util.Lists; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableFactory.UPSERT_DISABLED; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * This test aims to verify that {@link PulsarTableFactory} can consume proper config options and + * produce expected {@link PulsarTableSource} and {@link PulsarTableSink}. It guarantees that config + * options is used internally by the implementation classes. + */ +public class PulsarTableFactoryTest { + private static final String TEST_TOPIC = "test-topic"; + private static final String TEST_ADMIN_URL = "http://my-broker.example.com:8080"; + private static final String TEST_SERVICE_URL = "pulsar://localhost:6650"; + private static final String TEST_SUBSCRIPTION_NAME = "default-subscription"; + + private static final String NAME = "name"; + private static final String COUNT = "count"; + private static final String TIME = "time"; + private static final String METADATA = "metadata"; + private static final String WATERMARK_EXPRESSION = TIME + " - INTERVAL '5' SECOND"; + private static final DataType WATERMARK_DATATYPE = DataTypes.TIMESTAMP(3); + private static final String COMPUTED_COLUMN_NAME = "computed-column"; + private static final String COMPUTED_COLUMN_EXPRESSION = COUNT + " + 1.0"; + private static final DataType COMPUTED_COLUMN_DATATYPE = DataTypes.DECIMAL(10, 3); + + private static final Properties EXPECTED_PULSAR_SOURCE_PROPERTIES = new Properties(); + private static final Properties EXPECTED_PULSAR_SINK_PROPERTIES = new Properties(); + + private static final String FORMAT_DELIMITER_KEY = + String.format("%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()); + + private static final String FORMAT_FAIL_ON_MISSING_KEY = + String.format( + "%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.FAIL_ON_MISSING.key()); + + static { + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty(PULSAR_ADMIN_URL.key(), TEST_ADMIN_URL); + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty(PULSAR_SERVICE_URL.key(), TEST_SERVICE_URL); + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty( + PULSAR_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + + EXPECTED_PULSAR_SINK_PROPERTIES.setProperty(PULSAR_ADMIN_URL.key(), TEST_ADMIN_URL); + EXPECTED_PULSAR_SINK_PROPERTIES.setProperty(PULSAR_SERVICE_URL.key(), TEST_SERVICE_URL); + } + + private static final ResolvedSchema SCHEMA = + new ResolvedSchema( + Arrays.asList( + Column.physical(NAME, DataTypes.STRING().notNull()), + Column.physical(COUNT, DataTypes.DECIMAL(38, 18)), + Column.physical(TIME, DataTypes.TIMESTAMP(3)), + Column.computed( + COMPUTED_COLUMN_NAME, + ResolvedExpressionMock.of( + COMPUTED_COLUMN_DATATYPE, COMPUTED_COLUMN_EXPRESSION))), + Collections.singletonList( + WatermarkSpec.of( + TIME, + ResolvedExpressionMock.of( + WATERMARK_DATATYPE, WATERMARK_EXPRESSION))), + null); + + private static final ResolvedSchema SCHEMA_WITH_METADATA = + new ResolvedSchema( + Arrays.asList( + Column.physical(NAME, DataTypes.STRING()), + Column.physical(COUNT, DataTypes.DECIMAL(38, 18)), + Column.metadata(TIME, DataTypes.TIMESTAMP(3), "publish_time", false), + Column.metadata( + METADATA, DataTypes.STRING(), "value.metadata_2", false)), + Collections.emptyList(), + null); + + private static final DataType SCHEMA_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); + + @Test + public void testTableSource() { + final Map configuration = getBasicSourceOptions(); + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA, configuration); + + final DecodingFormat> valueDecodingFormat = + new TestFormatFactory.DecodingFormatMock(",", true); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_DATA_TYPE, + null, + new int[0], + valueDecodingFormat, + new int[] {0, 1, 2}, + UPSERT_DISABLED); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + valueDecodingFormat, + valueDecodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + StopCursor.never(), + SubscriptionType.Exclusive); + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + + ScanTableSource.ScanRuntimeProvider provider = + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertPulsarSourceIsSameAsExpected(provider); + } + + @Test + public void testTableSourceWithKeyValue() { + final Map configuration = getSourceKeyValueOptions(); + + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA, configuration); + // initialize stateful testing formats + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + + final TestFormatFactory.DecodingFormatMock keyDecodingFormat = + new TestFormatFactory.DecodingFormatMock("#", false); + keyDecodingFormat.producedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull(); + + final TestFormatFactory.DecodingFormatMock valueDecodingFormat = + new TestFormatFactory.DecodingFormatMock("|", false); + valueDecodingFormat.producedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3))) + .notNull(); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_DATA_TYPE, + keyDecodingFormat, + new int[] {0}, + valueDecodingFormat, + new int[] {1, 2}, + UPSERT_DISABLED); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + valueDecodingFormat, + valueDecodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + StopCursor.never(), + SubscriptionType.Exclusive); + + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + } + + @Test + public void testTableSourceWithKeyValueAndMetadata() { + final Map options = getSourceKeyValueOptions(); + options.put("test-format.readable-metadata", "metadata_1:INT, metadata_2:STRING"); + + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA_WITH_METADATA, options); + // initialize stateful testing formats + actualPulsarSource.applyReadableMetadata( + Arrays.asList("publish_time", "value.metadata_2"), + SCHEMA_WITH_METADATA.toSourceRowDataType()); + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + + final TestFormatFactory.DecodingFormatMock expectedKeyFormat = + new TestFormatFactory.DecodingFormatMock( + "#", false, ChangelogMode.insertOnly(), Collections.emptyMap()); + expectedKeyFormat.producedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING())).notNull(); + + final Map expectedReadableMetadata = new HashMap<>(); + expectedReadableMetadata.put("metadata_1", DataTypes.INT()); + expectedReadableMetadata.put("metadata_2", DataTypes.STRING()); + + final TestFormatFactory.DecodingFormatMock expectedValueFormat = + new TestFormatFactory.DecodingFormatMock( + "|", false, ChangelogMode.insertOnly(), expectedReadableMetadata); + expectedValueFormat.producedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD("metadata_2", DataTypes.STRING())) + .notNull(); + expectedValueFormat.metadataKeys = Collections.singletonList("metadata_2"); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_WITH_METADATA.toPhysicalRowDataType(), + expectedKeyFormat, + new int[] {0}, + expectedValueFormat, + new int[] {1}, + UPSERT_DISABLED); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + expectedValueFormat, + expectedValueFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + StopCursor.never(), + SubscriptionType.Exclusive); + + deserializationSchemaFactory.setProducedDataType( + SCHEMA_WITH_METADATA.toSourceRowDataType()); + deserializationSchemaFactory.setConnectorMetadataKeys( + Collections.singletonList("publish_time")); + + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + } + + @Test + public void testTableSink() { + final Map modifiedOptions = getBasicSinkOptions(); + final DynamicTableSink actualPulsarTableSink = createTableSink(SCHEMA, modifiedOptions); + + final EncodingFormat> valueEncodingFormat = + new TestFormatFactory.EncodingFormatMock(","); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + SCHEMA_DATA_TYPE, + null, + new int[0], + valueEncodingFormat, + new int[] {0, 1, 2}, + UPSERT_DISABLED); + + final PulsarTableSink expectedPulsarTableSink = + new PulsarTableSink( + serializationSchemaFactory, + valueEncodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SINK_PROPERTIES, + DeliveryGuarantee.AT_LEAST_ONCE, + null, + TopicRoutingMode.ROUND_ROBIN, + 0); + assertThat(actualPulsarTableSink).isEqualTo(expectedPulsarTableSink); + + DynamicTableSink.SinkRuntimeProvider provider = + actualPulsarTableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + assertThat(provider).isInstanceOf(SinkV2Provider.class); + final SinkV2Provider sinkProvider = (SinkV2Provider) provider; + final Sink sinkFunction = sinkProvider.createSink(); + assertThat(sinkFunction).isInstanceOf(PulsarSink.class); + } + + @Test + public void testTableSinkWithKeyValue() { + final Map modifiedOptions = getSinkKeyValueOptions(); + final PulsarTableSink actualPulsarTableSink = + (PulsarTableSink) createTableSink(SCHEMA, modifiedOptions); + // initialize stateful testing formats + actualPulsarTableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + + final TestFormatFactory.EncodingFormatMock keyEncodingFormat = + new TestFormatFactory.EncodingFormatMock("#"); + keyEncodingFormat.consumedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull(); + + final TestFormatFactory.EncodingFormatMock valueEncodingFormat = + new TestFormatFactory.EncodingFormatMock("|"); + valueEncodingFormat.consumedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3))) + .notNull(); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + SCHEMA_DATA_TYPE, + keyEncodingFormat, + new int[] {0}, + valueEncodingFormat, + new int[] {1, 2}, + UPSERT_DISABLED); + + final PulsarTableSink expectedPulsarTableSink = + new PulsarTableSink( + serializationSchemaFactory, + valueEncodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SINK_PROPERTIES, + DeliveryGuarantee.AT_LEAST_ONCE, + null, + TopicRoutingMode.ROUND_ROBIN, + 0); + assertThat(actualPulsarTableSink).isEqualTo(expectedPulsarTableSink); + } + + private static Map getBasicSourceOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + tableOptions.put(SOURCE_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, ","); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "true"); + return tableOptions; + } + + private static Map getSourceKeyValueOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + tableOptions.put(SOURCE_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, "|"); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "false"); + + tableOptions.put(KEY_FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put("key." + FORMAT_DELIMITER_KEY, "#"); + tableOptions.put("key." + FORMAT_FAIL_ON_MISSING_KEY, "false"); + tableOptions.put(KEY_FIELDS.key(), NAME); + return tableOptions; + } + + private static Map getBasicSinkOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, ","); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "true"); + return tableOptions; + } + + private static Map getSinkKeyValueOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, "|"); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "false"); + + tableOptions.put(KEY_FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put("key." + FORMAT_DELIMITER_KEY, "#"); + tableOptions.put("key." + FORMAT_FAIL_ON_MISSING_KEY, "false"); + tableOptions.put(KEY_FIELDS.key(), NAME); + return tableOptions; + } + + private PulsarSource assertPulsarSourceIsSameAsExpected( + ScanTableSource.ScanRuntimeProvider provider) { + assertThat(provider).isInstanceOf(SourceProvider.class); + final SourceProvider sourceProvider = (SourceProvider) provider; + return (PulsarSource) sourceProvider.createSource(); + } +} From e9749fb7e9725caa58ef32e393b6fa97a51483e6 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:30:25 +0800 Subject: [PATCH 07/20] g. option util and validation logic tests --- .../table/PulsarTableOptionUtilsTest.java | 416 +++++++++++++++ .../pulsar/table/PulsarTableOptionsTest.java | 496 ++++++++++++++++++ .../table/PulsarTableValidationUtilsTest.java | 188 +++++++ 3 files changed, 1100 insertions(+) create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java new file mode 100644 index 0000000000000..53ce6bd02a2a5 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java @@ -0,0 +1,416 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.PublishTimestampStopCursor; +import org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.utils.FactoryMocks; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createKeyFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createValueFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseAfterMessageIdStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseAtMessageIdStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdString; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** + * Unit test for {@link PulsarTableOptionUtils}. Tests each method and different inputs. Some tests + * have overlapping semantics with {@link PulsarTableOptionsTest} and {@link + * PulsarTableValidationUtilsTest}, but they cover different aspects of the validation, so all of + * them should be kept. + */ +public class PulsarTableOptionUtilsTest { + // -------------------------------------------------------------------------------------------- + // Format and Projection Test + // -------------------------------------------------------------------------------------------- + @Test + void formatProjection() { + final DataType dataType = + DataTypes.ROW( + FIELD("id", INT()), + FIELD("name", STRING()), + FIELD("age", INT()), + FIELD("address", STRING())); + + final Map options = createTestOptions(); + options.put("key.fields", "address; name"); + + final Configuration config = Configuration.fromMap(options); + + assertThat(createKeyFormatProjection(config, dataType)).containsExactly(3, 1); + assertThat(createValueFormatProjection(config, dataType)).containsExactly(0, 2); + } + + @Test + void invalidKeyFormatFieldProjection() { + final DataType dataType = ROW(FIELD("id", INT()), FIELD("name", STRING())); + final Map options = createTestOptions(); + options.put("key.fields", "non_existing"); + + final Configuration config = Configuration.fromMap(options); + + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> createKeyFormatProjection(config, dataType)) + .withMessage( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option: [id, name]", + "non_existing", KEY_FIELDS.key())); + } + + static Map createTestOptions() { + final Map options = new HashMap<>(); + options.put("key.format", "test-format"); + options.put("key.test-format.delimiter", ","); + options.put("value.format", "test-format"); + options.put("value.test-format.delimiter", "|"); + options.put("value.test-format.fail-on-missing", "true"); + return options; + } + + // -------------------------------------------------------------------------------------------- + // Table Source Option Utils Test + // -------------------------------------------------------------------------------------------- + + @Test + void topicsList() { + final Map options = createDefaultOptions(); + options.put(TOPICS.key(), ";"); + List topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).isEmpty(); + + options.put(TOPICS.key(), "topic1;"); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).hasSize(1); + + options.put(TOPICS.key(), "topic1;topic2"); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).hasSize(2); + + options.put(TOPICS.key(), ""); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).isEmpty(); + } + + @Test + void pulsarProperties() { + final Map options = createDefaultOptions(); + options.put(PULSAR_STATS_INTERVAL_SECONDS.key(), "30"); + Properties properties = getPulsarProperties(Configuration.fromMap(options)); + assertThat(properties.getProperty(PULSAR_STATS_INTERVAL_SECONDS.key())).isEqualTo("30"); + } + + @Test + void startCursor() { + // TDOO Use isEqualTo() to assert; need equals() method + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "other"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("MessageId format must be ledgerId:entryId:partitionId."); + + options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage( + "Could not parse value '123545L' for key 'source.start.publish-time'."); + } + + @Test + void subscriptionType() { + final Map options = createDefaultOptions(); + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Shared"); + SubscriptionType subscriptionType = getSubscriptionType(Configuration.fromMap(options)); + assertThat(subscriptionType).isEqualTo(SubscriptionType.Shared); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + subscriptionType = getSubscriptionType(Configuration.fromMap(options)); + assertThat(subscriptionType).isEqualTo(SubscriptionType.Exclusive); + } + + @Test + void canParseMessageIdEarliestOrLatestStartCursor() { + String earliest = "earliest"; + StartCursor startCursor = parseMessageIdStartCursor(earliest); + assertThat(startCursor).isEqualTo(StartCursor.earliest()); + + String latest = "latest"; + startCursor = parseMessageIdStartCursor(latest); + assertThat(startCursor).isEqualTo(StartCursor.latest()); + + String precise = "0:0:100"; + startCursor = parseMessageIdStartCursor(precise); + assertThat(startCursor).isEqualTo(StartCursor.fromMessageId(new MessageIdImpl(0, 0, 100))); + } + + @Test + void publishTimeStartCursor() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("Could not parse value '12345L' for key 'source.start.publish-time'."); + } + + @Test + void canParseMessageIdNeverOrLatestStopCursor() { + String never = "never"; + StopCursor stopCursor = parseAtMessageIdStopCursor(never); + assertThat(stopCursor).isEqualTo(StopCursor.never()); + + String latest = "latest"; + stopCursor = parseAtMessageIdStopCursor(latest); + assertThat(stopCursor).isEqualTo(StopCursor.latest()); + + String precise = "0:0:100"; + stopCursor = parseAtMessageIdStopCursor(precise); + assertThat(stopCursor).isEqualTo(StopCursor.atMessageId(new MessageIdImpl(0, 0, 100))); + + stopCursor = parseAfterMessageIdStopCursor(precise); + assertThat(stopCursor).isEqualTo(StopCursor.afterMessageId(new MessageIdImpl(0, 0, 100))); + } + + @Test + void publishTimeStopCursor() { + final Map options = createDefaultOptions(); + options.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "12345"); + StopCursor stopCursor = getStopCursor(Configuration.fromMap(options)); + assertThat(stopCursor).isInstanceOf(PublishTimestampStopCursor.class); + + options.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "12345L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStopCursor(Configuration.fromMap(options))) + .withMessage( + "Could not parse value '12345L' for key 'source.stop.at-publish-time'."); + } + + @Test + void canParseMessageIdUsingMessageIdImpl() { + final String invalidFormatMessage = + "MessageId format must be ledgerId:entryId:partitionId."; + final String invalidNumberMessage = + "MessageId format must be ledgerId:entryId:partitionId. Each id should be able to parsed to long type."; + String precise = "0:0:100"; + assertThatNoException().isThrownBy(() -> parseMessageIdString(precise)); + + String empty = ""; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(empty)) + .withMessage(invalidFormatMessage); + + String noSemicolon = "0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(noSemicolon)) + .withMessage(invalidFormatMessage); + + String oneSemiColon = "0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(oneSemiColon)) + .withMessage(invalidFormatMessage); + + String oneSemiColonComplete = "0:0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(oneSemiColonComplete)) + .withMessage(invalidFormatMessage); + + String twoSemiColon = "0:0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(twoSemiColon)) + .withMessage(invalidNumberMessage); + + String twoSemiColonComplete = "0:0:0"; + assertThatNoException().isThrownBy(() -> parseMessageIdString(twoSemiColonComplete)); + + String threeSemicolon = "0:0:0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(threeSemicolon)) + .withMessage(invalidNumberMessage); + + String threeSemicolonComplete = "0:0:0:0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(threeSemicolonComplete)) + .withMessage(invalidNumberMessage); + + String invalidNumber = "0:0:adf"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(invalidNumber)) + .withMessage(invalidNumberMessage); + } + + // -------------------------------------------------------------------------------------------- + // Table Sink Option Utils Test + // -------------------------------------------------------------------------------------------- + + @Test + void topicRouter() { + final Map options = createDefaultOptions(); + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter"); + TopicRouter topicRouter = + getTopicRouter(Configuration.fromMap(options), FactoryMocks.class.getClassLoader()); + assertThat(topicRouter).isInstanceOf(MockTopicRouter.class); + + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.PulsarTableOptionsTest"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + getTopicRouter( + Configuration.fromMap(options), + FactoryMocks.class.getClassLoader())) + .withMessage( + String.format( + "Sink TopicRouter class '%s' should extend from the required class %s", + PulsarTableOptionsTest.class.getName(), + TopicRouter.class.getName())); + + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.testutils.NonExistMockTopicRouter"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + getTopicRouter( + Configuration.fromMap(options), + FactoryMocks.class.getClassLoader())) + .withMessage( + String.format( + "Could not find and instantiate TopicRouter class '%s'", + "org.apache.flink.connector.pulsar.table.testutils.NonExistMockTopicRouter")); + } + + @Test + void topicRoutingMode() { + final Map options = createDefaultOptions(); + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + TopicRoutingMode topicRoutingMode = getTopicRoutingMode(Configuration.fromMap(options)); + assertThat(topicRoutingMode).isEqualTo(TopicRoutingMode.ROUND_ROBIN); + + options.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); + topicRoutingMode = getTopicRoutingMode(Configuration.fromMap(options)); + assertThat(topicRoutingMode).isEqualTo(TopicRoutingMode.MESSAGE_KEY_HASH); + } + + @Test + void messageDelayMillis() { + final Map options = createDefaultOptions(); + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10 s"); + long messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofSeconds(10).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofSeconds(10).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1000ms"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMillis(1000).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 d"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofDays(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 H"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofHours(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 min"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMinutes(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1m"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMinutes(1).toMillis()); + } + + private Map createDefaultOptions() { + Map optionMap = new HashMap<>(); + return optionMap; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java new file mode 100644 index 0000000000000..cebd6eafb8658 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java @@ -0,0 +1,496 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.connector.pulsar.table.testutils.MockPulsarAuthentication; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.factories.FactoryUtil; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.factories.TestDynamicTableFactory.VALUE_FORMAT; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** + * Test config options for Pulsar SQL connector. This test aims to verify legal combination of + * config options will be accepted and do not cause runtime exceptions (but cannot guarantee they + * are taking effect), and illegal combinations of config options will be rejected early. + */ +public class PulsarTableOptionsTest extends PulsarTableTestBase { + @Test + void noTopicsSpecified() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + "One or more required options are missing.\n\n" + + "Missing required options are:\n\n" + + "topics")); + } + + @Test + void invalidEmptyTopics() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + testConfigs.put(TOPICS.key(), ""); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, new ValidationException("The topics list should not be empty.")); + } + + @Test + void topicsWithSemicolon() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + testConfigs.put(TOPICS.key(), topicName + ";"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidTopicName() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + String invalidTopicName = "persistent://tenant/no-topic"; + testConfigs.put(TOPICS.key(), invalidTopicName); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + String.format( + "The topics name %s is not a valid topic name.", + invalidTopicName))); + } + + @Test + void topicsList() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + testConfigs.put( + TOPICS.key(), + topicNameWithPartition(topicName, 0) + ";" + topicNameWithPartition(topicName, 1)); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void usingFormat() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void usingValueFormat() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(VALUE_FORMAT.key(), "json"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void usingValueFormatAndFormatOptions() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(VALUE_FORMAT.key(), "json"); + testConfigs.put("value.json.fail-on-missing-field", "false"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void subscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidUnsupportedSubscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Failover"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, new ValidationException("Failover SubscriptionType is not supported. ")); + } + + @Test + void invalidNonExistSubscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "random-subscription-type"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException("Invalid value for option 'source.subscription-type'.")); + } + + @Test + void messageIdStartCursorEarliest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStartCursorLatest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStartCursorExact() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidMessageIdStartCursorEmptyId() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId. " + + "Each id should be able to parsed to long type.")); + } + + @Test + void invalidMessageIdStartCursorIncomplete() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId.")); + } + + @Test + void timestampStartCursor() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "233010230"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStopCursorNever() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "never"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStopCursorLatest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "latest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStopCursorExact() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "0:0:-1"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void timestampStopCursor() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "233010230"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void topicRoutingMode() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidTopicRouter() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + String invalidClassName = "invalid class name"; + testConfigs.put(SINK_CUSTOM_TOPIC_ROUTER.key(), invalidClassName); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + String.format( + "Could not find and instantiate TopicRouter class '%s'", + invalidClassName))); + } + + @Test + void messageDelay() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidMessageDelay() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "invalid-duration"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException("Invalid value for option 'sink.message-delay-interval'.")); + } + + // -------------------------------------------------------------------------------------------- + // PulsarOptions, PulsarSourceOptions, PulsarSinkOptions Test + // -------------------------------------------------------------------------------------------- + + @Test + void pulsarOptionsAuthParamMap() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(PULSAR_AUTH_PARAM_MAP.key(), "key1:value1,key2:value2"); + testConfigs.put( + PULSAR_AUTH_PLUGIN_CLASS_NAME.key(), MockPulsarAuthentication.class.getName()); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + // -------------------------------------------------------------------------------------------- + // requiredOptions(), optionalOptions() Test + // -------------------------------------------------------------------------------------------- + + @Test + void unusedConfigOptions() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put("random_config", "random_value"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException(topicName, ValidationException.class); + } + + // -------------------------------------------------------------------------------------------- + // Utils methods + // -------------------------------------------------------------------------------------------- + + private String createTestConfig(Map configMap) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : configMap.entrySet()) { + sb.append(String.format(" '%s' = '%s' ,\n", entry.getKey(), entry.getValue())); + } + return sb.toString(); + } + + private void runSql(String topicName, String testConfigString) { + createTestTopic(topicName, 2); + final String createTable = + String.format( + "CREATE TABLE %s (\n" + + " `physical_1` STRING,\n" + + " `physical_2` INT,\n" + + " `physical_3` BOOLEAN\n" + + ") WITH (\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " %s\n" + + " 'connector' = 'pulsar'" + + ")", + topicName, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + testConfigString); + tableEnv.executeSql(createTable); + } + + private void runSinkAndExpectSucceed(String topicName) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatNoException().isThrownBy(() -> tableEnv.executeSql(initialValues).await()); + } + + private void runSinkAndExpectException( + String topicName, final Class exceptionType) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.executeSql(initialValues).await()); + } + + private void runSinkAndExpectException(String topicName, Throwable cause) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.executeSql(initialValues).await()) + .withCause(cause); + } + + private void runSourceAndExpectSucceed(String topicName) { + assertThatNoException() + .isThrownBy(() -> tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName))); + } + + private void runSourceAndExpectException(String topicName, Throwable cause) { + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName))) + .withCause(cause); + } + + private String randomTopicName() { + final String testTopicPrefix = "test_config_topic"; + return testTopicPrefix + randomAlphabetic(5); + } + + private Map testConfigWithTopicAndFormat(String tableName) { + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), tableName); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + return testConfigs; + } + + private Map testConfigWithFormat() { + Map testConfigs = new HashMap<>(); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + return testConfigs; + } + + private Map testConfigWithTopic(String tableName) { + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), tableName); + return testConfigs; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java new file mode 100644 index 0000000000000..ca8eb663158db --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java @@ -0,0 +1,188 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.ValidationException; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateKeyFormatConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateSinkRoutingConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateStartCursorConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateSubscriptionTypeConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTopicsConfigs; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** Unit test for {@link PulsarTableValidationUtils}. */ +public class PulsarTableValidationUtilsTest extends PulsarTableTestBase { + @Test + void topicsConfigs() { + final Map options = createDefaultOptions(); + options.put(TOPICS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage("The topics list should not be empty."); + + String invalidTopicName = "persistent://tenant/topic"; + String validTopicName = "valid-topic"; + + options.put(TOPICS.key(), invalidTopicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The topics name %s is not a valid topic name.", invalidTopicName)); + + options.put(TOPICS.key(), validTopicName + ";" + invalidTopicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The topics name %s is not a valid topic name.", invalidTopicName)); + + options.put(TOPICS.key(), validTopicName + ";" + validTopicName); + assertThatNoException() + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))); + + options.put(TOPICS.key(), validTopicName + ";"); + assertThatNoException() + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))); + } + + @Test + void startCursorConfigs() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + assertThatNoException() + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))); + + options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "2345123234"); + assertThatNoException() + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "2345123234"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME)); + } + + @Test + void subscriptionTypeConfigs() { + final Map options = createDefaultOptions(); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Shared"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Key_Shared"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "invalid-subscription"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))) + .withMessage( + "Could not parse value 'invalid-subscription' for key 'source.subscription-type'."); + } + + @Test + void sinkRoutingConfigs() { + final Map options = createDefaultOptions(); + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + assertThatNoException() + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))); + + // validation does not try to create the class + options.remove(SINK_TOPIC_ROUTING_MODE.key()); + options.put(SINK_CUSTOM_TOPIC_ROUTER.key(), "invalid-class-name"); + assertThatNoException() + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))); + + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + options.put(SINK_CUSTOM_TOPIC_ROUTER.key(), "invalid-class-name"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE)); + } + + @Test + void keyFormatConfigs() { + final Map options = createDefaultOptions(); + options.put(KEY_FIELDS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.remove(KEY_FIELDS.key()); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.put(KEY_FIELDS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.put(KEY_FIELDS.key(), "k_field1"); + assertThatNoException() + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))); + } + + private Map createDefaultOptions() { + Map optionMap = new HashMap<>(); + return optionMap; + } +} From 688588fc0f30cf40fdc8826a3d5e96fce107ccea Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:44:22 +0800 Subject: [PATCH 08/20] h. pom and manifest file changes --- .../flink-connector-pulsar/pom.xml | 90 +++++++++++++++++++ .../org.apache.flink.table.factories.Factory | 16 ++++ 2 files changed, 106 insertions(+) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index d17b83ef974ec..cac2407b74ea5 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -55,6 +55,36 @@ under the License. ${project.version} + + + + org.apache.flink + flink-table-common + ${project.version} + provided + + + + org.apache.flink + flink-table-api-java-bridge + ${project.version} + provided + + + + org.apache.flink + flink-table-runtime + ${project.version} + provided + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + provided + + @@ -74,6 +104,22 @@ under the License. true + + + + org.apache.flink + flink-json + ${project.version} + provided + + + + org.apache.flink + flink-avro + ${project.version} + provided + + @@ -99,6 +145,50 @@ under the License. test + + + org.apache.flink + flink-table-common + ${project.version} + test-jar + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-table-api-scala-bridge_${scala.binary.version} + ${project.version} + test + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-avro-confluent-registry + ${project.version} + test + + + org.apache.flink + flink-csv + ${project.version} + test + + diff --git a/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000000..b153b56175670 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.connector.pulsar.table.PulsarTableFactory From 0db13ea451d5dd62ee32a196ddd5a51ad336af05 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 08:51:21 +0800 Subject: [PATCH 09/20] i. doc configuration changes --- .../generated/pulsar_table_configuration.html | 90 +++++++++++++++++++ .../docs/util/ConfigurationOptionLocator.java | 3 + 2 files changed, 93 insertions(+) create mode 100644 docs/layouts/shortcodes/generated/pulsar_table_configuration.html diff --git a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html new file mode 100644 index 0000000000000..c84bc6bb1382c --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html @@ -0,0 +1,90 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
admin-url
(none)StringThe Pulsar service HTTP URL for the admin endpoint. For example, http://my-broker.example.com:8080, or https://my-broker.example.com:8443 for TLS.
explicit
trueBooleanIndicate if the table is an explicit Flink table.
key.fields
List<String>An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined.
key.format
(none)StringDefines the format identifier for decoding/encoding key bytes in Pulsar message. The identifier is used to discover a suitable format factory.
service-url
(none)StringService URL provider for Pulsar service.
To connect to Pulsar using client libraries, you need to specify a Pulsar protocol URL.
You can assign Pulsar protocol URLs to specific clusters and use the Pulsar scheme.
  • This is an example of localhost: pulsar://localhost:6650.
  • If you have multiple brokers, the URL is as: pulsar://localhost:6550,localhost:6651,localhost:6652
  • A URL for a production Pulsar cluster is as: pulsar://pulsar.us-west.example.com:6650
  • If you use TLS authentication, the URL is as pulsar+ssl://pulsar.us-west.example.com:6651
sink.custom-topic-router
(none)String(Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the sink.topic-routing-mode option will be ignored.
sink.message-delay-interval
0 msDuration(Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink).
sink.topic-routing-mode
round-robin

Enum

(Optional) the topic routing mode. Available options are round-robin and message-key-hash. By default, it is set to round-robin. If you want to use a custom topic router, use the sink.custom-topic-router option to determine the partition for a particular message.

Possible values:
  • "round-robin": The producer will publish messages across all partitions in a round-robin fashion to achieve maximum throughput. Please note that round-robin is not done per individual message but rather it's set to the same boundary of pulsar.producer.batchingMaxMessages, to ensure batching is effective.
  • "message-key-hash": If no key is provided, The partitioned producer will randomly pick one single topic partition and publish all the messages into that partition. If a key is provided on the message, the partitioned producer will hash the key and assign the message to a particular partition.
  • "custom": Use custom TopicRouter implementation that will be called to determine the partition for a particular message.
source.start.message-id
(none)StringOptional message id used to specify a consuming starting point for source. Use earliest, latest or pass in a message id representation in ledgerId:entryId:partitionId, such as 12:2:-1
source.start.publish-time
(none)Long(Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data.
source.subscription-name
"flink-sql-connector-pulsar"StringThe subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer.
source.subscription-type
Exclusive

Enum

The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only Exclusive and Shared subscription types are supported.

Possible values:
  • "Exclusive"
  • "Shared"
  • "Failover"
  • "Key_Shared"
topics
(none)List<String>Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (;) like topic-1;topic-2.
diff --git a/flink-docs/src/main/java/org/apache/flink/docs/util/ConfigurationOptionLocator.java b/flink-docs/src/main/java/org/apache/flink/docs/util/ConfigurationOptionLocator.java index 8ea9bb711d6c6..7e16d4e47b3ea 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/util/ConfigurationOptionLocator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/util/ConfigurationOptionLocator.java @@ -82,6 +82,9 @@ public class ConfigurationOptionLocator { new OptionsClassLocation( "flink-connectors/flink-connector-pulsar", "org.apache.flink.connector.pulsar.sink"), + new OptionsClassLocation( + "flink-connectors/flink-connector-pulsar", + "org.apache.flink.connector.pulsar.table"), new OptionsClassLocation( "flink-libraries/flink-cep", "org.apache.flink.cep.configuration"), new OptionsClassLocation( From bf20d2acf0660e350319457153de18f5adfc2465 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 11:48:43 +0800 Subject: [PATCH 10/20] j. DataStream: stopCursor add equals() --- .../cursor/stop/EventTimestampStopCursor.java | 19 +++++++++++++++++++ .../cursor/stop/LatestMessageStopCursor.java | 19 +++++++++++++++++++ .../cursor/stop/MessageIdStopCursor.java | 19 +++++++++++++++++++ .../cursor/stop/NeverStopCursor.java | 11 +++++++++++ .../stop/PublishTimestampStopCursor.java | 19 +++++++++++++++++++ 5 files changed, 87 insertions(+) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java index d2a44ea362d80..df5dee80d0c9c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java @@ -22,6 +22,8 @@ import org.apache.pulsar.client.api.Message; +import java.util.Objects; + /** Stop consuming message at the given event time. */ public class EventTimestampStopCursor implements StopCursor { private static final long serialVersionUID = 2391576769339369027L; @@ -39,4 +41,21 @@ public StopCondition shouldStop(Message message) { long eventTime = message.getEventTime(); return StopCondition.compare(timestamp, eventTime, inclusive); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EventTimestampStopCursor that = (EventTimestampStopCursor) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java index 0de963ed4fb53..0311e4a8fb362 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java @@ -25,6 +25,8 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import java.util.Objects; + import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; /** @@ -55,4 +57,21 @@ public void open(PulsarAdmin admin, TopicPartition partition) { this.messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic)); } } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LatestMessageStopCursor that = (LatestMessageStopCursor) o; + return inclusive == that.inclusive && Objects.equals(messageId, that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId, inclusive); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java index 03d83aa4495d1..4532e45df6e3d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java @@ -24,6 +24,8 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.MessageIdImpl; +import java.util.Objects; + import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.pulsar.client.api.MessageId.earliest; @@ -56,4 +58,21 @@ public StopCondition shouldStop(Message message) { MessageId current = message.getMessageId(); return StopCondition.compare(messageId, current, inclusive); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MessageIdStopCursor that = (MessageIdStopCursor) o; + return inclusive == that.inclusive && messageId.equals(that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId, inclusive); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java index 3eb035634ae0a..27dfc6e30eff8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java @@ -30,4 +30,15 @@ public class NeverStopCursor implements StopCursor { public StopCondition shouldStop(Message message) { return StopCondition.CONTINUE; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + return true; + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java index 2dfdd7658428a..cca722dd83d9c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java @@ -22,6 +22,8 @@ import org.apache.pulsar.client.api.Message; +import java.util.Objects; + /** Stop consuming message at the given publish time. */ public class PublishTimestampStopCursor implements StopCursor { private static final long serialVersionUID = 4386276745339324527L; @@ -39,4 +41,21 @@ public StopCondition shouldStop(Message message) { long publishTime = message.getPublishTime(); return StopCondition.compare(timestamp, publishTime, inclusive); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PublishTimestampStopCursor that = (PublishTimestampStopCursor) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp); + } } From 7ce3909249fdcf573400f861ecf4dcecb539c84c Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 11:51:09 +0800 Subject: [PATCH 11/20] k. DataStream: add keyBytes() method for writer --- .../pulsar/sink/writer/PulsarWriter.java | 17 +++++++++++++++-- .../sink/writer/message/PulsarMessage.java | 8 ++++++++ .../writer/message/PulsarMessageBuilder.java | 11 +++++++++++ 3 files changed, 34 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java index 927e40c31145f..fe223cbb24766 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java @@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Base64; import java.util.Collection; import java.util.List; import java.util.Map; @@ -133,9 +134,16 @@ public void write(IN element, Context context) throws IOException, InterruptedEx PulsarMessage message = serializationSchema.serialize(element, sinkContext); // Choose the right topic to send. - String key = message.getKey(); + List availableTopics = metadataListener.availableTopics(); - String topic = topicRouter.route(element, key, availableTopics, sinkContext); + String keyString; + // TODO if both keyBytes and key are set, use keyBytes. This is a temporary solution. + if (message.getKeyBytes() == null) { + keyString = message.getKey(); + } else { + keyString = Base64.getEncoder().encodeToString(message.getKeyBytes()); + } + String topic = topicRouter.route(element, keyString, availableTopics, sinkContext); // Create message builder for sending message. TypedMessageBuilder builder = createMessageBuilder(topic, context, message); @@ -206,6 +214,11 @@ private TypedMessageBuilder createMessageBuilder( builder.key(key); } + byte[] keyBytes = message.getKeyBytes(); + if (keyBytes != null) { + builder.keyBytes(keyBytes); + } + long eventTime = message.getEventTime(); if (eventTime > 0) { builder.eventTime(eventTime); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java index 0c45763cdb1ef..0dd52655f9040 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java @@ -37,6 +37,7 @@ public class PulsarMessage { @Nullable private final byte[] orderingKey; @Nullable private final String key; + @Nullable private final byte[] keyBytes; private final long eventTime; private final Schema schema; @Nullable private final T value; @@ -49,6 +50,7 @@ public class PulsarMessage { PulsarMessage( @Nullable byte[] orderingKey, @Nullable String key, + @Nullable byte[] keyBytes, long eventTime, Schema schema, @Nullable T value, @@ -58,6 +60,7 @@ public class PulsarMessage { boolean disableReplication) { this.orderingKey = orderingKey; this.key = key; + this.keyBytes = keyBytes; this.eventTime = eventTime; this.schema = schema; this.value = value; @@ -77,6 +80,11 @@ public String getKey() { return key; } + @Nullable + public byte[] getKeyBytes() { + return keyBytes; + } + public long getEventTime() { return eventTime; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java index 518593282ef78..8794c212f668f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java @@ -36,6 +36,7 @@ public class PulsarMessageBuilder { private byte[] orderingKey; private String key; + private byte[] keyBytes; private long eventTime; Schema schema; private T value; @@ -59,6 +60,15 @@ public PulsarMessageBuilder key(String key) { return this; } + /** + * Property {@link TypedMessageBuilder#keyBytes(byte[])}. This property would also be used in + * {@link KeyHashTopicRouter}. + */ + public PulsarMessageBuilder keyBytes(byte[] keyBytes) { + this.keyBytes = checkNotNull(keyBytes); + return null; + } + /** Method wrapper of {@link TypedMessageBuilder#eventTime(long)}. */ public PulsarMessageBuilder eventTime(long eventTime) { this.eventTime = eventTime; @@ -116,6 +126,7 @@ public PulsarMessage build() { return new PulsarMessage<>( orderingKey, key, + keyBytes, eventTime, schema, value, From d69b29c917ccf378639a4a88403e326ad9506801 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 11:57:11 +0800 Subject: [PATCH 12/20] l. DataStream: add typeInformation for Schema Factory --- .../common/schema/factories/AvroSchemaFactory.java | 13 +++++++++++++ .../common/schema/factories/JSONSchemaFactory.java | 13 +++++++++++++ .../schema/factories/KeyValueSchemaFactory.java | 7 ++----- .../schema/factories/JSONSchemaFactoryTest.java | 4 ++-- 4 files changed, 30 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java index ab97b4102cc31..8288136288641 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.pulsar.common.schema.factories; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.AvroUtils; + import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.schema.AvroSchema; @@ -45,4 +48,14 @@ public Schema createSchema(SchemaInfo info) { return AvroSchema.of(definition); } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + try { + Class decodeClassInfo = decodeClassInfo(info); + return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); + } catch (Exception e) { + return super.createTypeInfo(info); + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java index 8d1a4944d0c5c..2280d353d95af 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.pulsar.common.schema.factories; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.AvroUtils; + import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.common.schema.SchemaInfo; @@ -38,4 +41,14 @@ public Schema createSchema(SchemaInfo info) { Class typeClass = decodeClassInfo(info); return JSONSchema.of(typeClass, info.getProperties()); } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + try { + Class decodeClassInfo = decodeClassInfo(info); + return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); + } catch (Exception e) { + return super.createTypeInfo(info); + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java index 893e7e6eeb1db..d60f06d2a7643 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java @@ -69,11 +69,8 @@ public Schema> createSchema(SchemaInfo info) { public TypeInformation> createTypeInfo(SchemaInfo info) { KeyValue kvSchemaInfo = decodeKeyValueSchemaInfo(info); - Schema keySchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getKey()); - Class keyClass = decodeClassInfo(keySchema.getSchemaInfo()); - - Schema valueSchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getValue()); - Class valueClass = decodeClassInfo(valueSchema.getSchemaInfo()); + Class keyClass = decodeClassInfo(kvSchemaInfo.getKey()); + Class valueClass = decodeClassInfo(kvSchemaInfo.getValue()); Schema> schema = createSchema(info); PulsarSchema> pulsarSchema = diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java index e4b2dc1b9e9cf..d67b64ceac9e7 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; -import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaTypeInformation; import org.apache.flink.connector.pulsar.testutils.SampleData.FL; import org.apache.flink.connector.pulsar.testutils.SampleData.Foo; +import org.apache.flink.formats.avro.typeutils.AvroTypeInfo; import org.apache.flink.util.InstantiationUtil; import org.apache.pulsar.client.api.Schema; @@ -62,7 +62,7 @@ void createJSONTypeInformationFromSchemaInfo() { TypeInformation typeInfo = factory.createTypeInfo(pulsarSchema.getSchemaInfo()); assertThat(typeInfo) - .isInstanceOf(PulsarSchemaTypeInformation.class) + .isInstanceOf(AvroTypeInfo.class) .hasFieldOrPropertyWithValue("typeClass", FL.class); // TypeInformation serialization. From 7f1973401802739dcd85f1083ce383f9df12e7ce Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 12:00:41 +0800 Subject: [PATCH 13/20] m. DataStream: startCursor add equals() --- .../cursor/start/MessageIdStartCursor.java | 18 ++++++++++++++++ .../cursor/start/TimestampStartCursor.java | 21 ++++++++++++++++++- 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index 03d5ba3e1339e..5a2e1f615b17c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -27,6 +27,7 @@ import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId; import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; +import java.util.Objects; /** This cursor would leave pulsar start consuming from a specific message id. */ public class MessageIdStartCursor implements StartCursor { @@ -61,4 +62,21 @@ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { public CursorPosition position(String topic, int partitionId) { return new CursorPosition(messageId); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MessageIdStartCursor that = (MessageIdStartCursor) o; + return Objects.equals(messageId, that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java index da51a58e9433c..ee1d43723a414 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java @@ -21,7 +21,9 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; -/** This cursor would left pulsar start consuming from a specific publish timestamp. */ +import java.util.Objects; + +/** A cursor starts consuming from a specific publish timestamp. */ public class TimestampStartCursor implements StartCursor { private static final long serialVersionUID = 5170578885838095320L; @@ -35,4 +37,21 @@ public TimestampStartCursor(long timestamp, boolean inclusive) { public CursorPosition position(String topic, int partitionId) { return new CursorPosition(timestamp); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TimestampStartCursor that = (TimestampStartCursor) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp); + } } From 8d4b27223f0d524daf544e9569a5caa4c677c6e0 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 12:08:31 +0800 Subject: [PATCH 14/20] n. apply checkstyle and spotless --- .../enumerator/cursor/start/MessageIdStartCursor.java | 3 ++- .../source/PulsarTableDeserializationSchemaFactory.java | 4 ++-- .../connector/pulsar/table/PulsarTableFactoryTest.java | 8 ++++---- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index 5a2e1f615b17c..1f26e834749fa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -25,9 +25,10 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.MessageIdImpl; +import java.util.Objects; + import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId; import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; -import java.util.Objects; /** This cursor would leave pulsar start consuming from a specific message id. */ public class MessageIdStartCursor implements StartCursor { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java index ab9b6a0ba9386..70b7678d457ce 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java @@ -42,8 +42,8 @@ /** * Contains key, value projection and format information, and use such information to create a - * {@link PulsarTableDeserializationSchema} instance - * used by runtime {@link org.apache.flink.connector.pulsar.source.PulsarSource} instance. + * {@link PulsarTableDeserializationSchema} instance used by runtime {@link + * org.apache.flink.connector.pulsar.source.PulsarSource} instance. * *

A Flink row fields has a strict order: Physical Fields (Key + value) + Format Metadata Fields * Connector Metadata Fields. Physical Fields are fields come directly from Pulsar message body; diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java index 361fa9d00daf6..145571a918bd2 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java @@ -27,7 +27,6 @@ import org.apache.flink.connector.pulsar.source.PulsarSource; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.table.PulsarTableFactory; import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; @@ -77,9 +76,10 @@ import static org.assertj.core.api.Assertions.assertThat; /** - * This test aims to verify that {@link PulsarTableFactory} can consume proper config options and - * produce expected {@link PulsarTableSource} and {@link PulsarTableSink}. It guarantees that config - * options is used internally by the implementation classes. + * This test aims to verify that {@link org.apache.flink.connector.pulsar.table.PulsarTableFactory} + * can consume proper config options and produce expected {@link PulsarTableSource} and {@link + * PulsarTableSink}. It guarantees that config options is used internally by the implementation + * classes. */ public class PulsarTableFactoryTest { private static final String TEST_TOPIC = "test-topic"; From 6ef2f615ff3aec90c9bc14f09b17523ec1cded9f Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 20 Sep 2022 12:42:19 +0800 Subject: [PATCH 15/20] o. DataStream: make one operator method createProducer() public --- .../pulsar/testutils/runtime/PulsarRuntimeOperator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java index 5d14d7ee08169..c4cca63187191 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java @@ -478,7 +478,7 @@ private void createPartitionedTopic(String topic, int numberOfPartitions) { } } - private Producer createProducer(String topic, Schema schema) { + public synchronized Producer createProducer(String topic, Schema schema) { ProducerBuilder builder = client().newProducer(schema) .topic(topic) From 8e6e3a36463bdd26a67758918a95de7d63762425 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Thu, 22 Sep 2022 18:43:26 +0800 Subject: [PATCH 16/20] p. set test default paralellism to 2 --- .../flink/connector/pulsar/table/PulsarTableTestBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java index 24a4e7b9c6e1f..beeceb8540e15 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -58,7 +58,7 @@ protected PulsarRuntime runtime() { return PulsarRuntime.container(); } - private static final int DEFAULT_PARALLELISM = 1; + private static final int DEFAULT_PARALLELISM = 2; @BeforeAll public void beforeAll() { From 23efa429263f9bb01e22e6615be89f1a69460c1f Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Fri, 23 Sep 2022 09:40:46 +0800 Subject: [PATCH 17/20] 1. fix unapproved license issue --- .../pulsar/table/testutils/TestingUser.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java index 1cedc2e5b9766..fa8365b78af91 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java @@ -1,3 +1,21 @@ +/* + * 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.connector.pulsar.table.testutils; import java.io.Serializable; @@ -8,7 +26,7 @@ /** * A test POJO class used by table integration tests to validate the JSON and AVRO schema are - * compatible with corresponding Flink formats + * compatible with corresponding Flink formats. */ public class TestingUser implements Serializable { private static final long serialVersionUID = -1123545861004770003L; From 21977c782760446e72f96ad7a609d412fd828bde Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Thu, 13 Oct 2022 21:39:04 +0800 Subject: [PATCH 18/20] 2. refactor ITCase to use MiniCluster --- .../generated/pulsar_table_configuration.html | 26 ++++++++++++- .../pulsar/table/PulsarTableITCase.java | 6 +-- .../pulsar/table/PulsarTableTestBase.java | 39 +++++++++++-------- 3 files changed, 49 insertions(+), 22 deletions(-) diff --git a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html index c84bc6bb1382c..f87fa1766d416 100644 --- a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html @@ -68,9 +68,27 @@ Long (Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. + +

source.stop.after-message-id
+ (none) + String + Optional message id used to specify a stop position but include the given message in the consuming result for the unbounded sql source. Pass in a message id representation in "ledgerId:entryId:partitionId", such as "12:2:-1". + + +
source.stop.at-message-id
+ (none) + String + Optional message id used to specify a stop cursor for the unbounded sql source. Use "never", "latest" or pass in a message id representation in "ledgerId:entryId:partitionId", such as "12:2:-1" + + +
source.stop.at-publish-time
+ (none) + Long + Optional publish timestamp used to specify a stop cursor for the unbounded sql source. +
source.subscription-name
- "flink-sql-connector-pulsar" + (none) String The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer. @@ -86,5 +104,11 @@ List<String> Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (;) like topic-1;topic-2. + +
value.format
+ (none) + String + Defines the format identifier for decoding/encoding value data. The identifier is used to discover a suitable format factory. + diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java index 4eada34491be6..67e076bbb7233 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -24,7 +24,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; @@ -32,8 +31,8 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -65,9 +64,7 @@ * IT cases for the Pulsar table source and sink. It aims to verify runtime behaviour and certain * use cases are correct and can produce/consume the desired records as user specifies. */ -@ExtendWith(MiniClusterExtension.class) public class PulsarTableITCase extends PulsarTableTestBase { - protected static final String JSON_FORMAT = "json"; protected static final String AVRO_FORMAT = "avro"; protected static final String CSV_FORMAT = "csv"; @@ -337,6 +334,7 @@ void sendMessageWithPropertiesAndReadPropertiesMetadata() throws Exception { @ParameterizedTest @MethodSource("provideSchemaData") + @Disabled() void readAndSelectIntoTableUsingSimpleSchema( String format, Schema schema, T value, String flinkTableDataType) throws Exception { final String sourceTopic = "source_topic_" + randomAlphanumeric(3); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java index beeceb8540e15..3140d48f267e7 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -21,17 +21,17 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; -import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; -import org.apache.flink.connector.testframe.junit.annotations.TestEnv; -import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; -import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; -import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,30 +40,35 @@ public abstract class PulsarTableTestBase { private static final Logger LOG = LoggerFactory.getLogger(PulsarTableTestBase.class); - @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + private static final int DEFAULT_PARALLELISM = 1; - // Defines pulsar running environment - @TestExternalSystem - protected PulsarTestEnvironment pulsar = new PulsarTestEnvironment(runtime()); + @RegisterExtension + protected static final MiniClusterExtension CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); - @TestSemantics - protected CheckpointingMode[] semantics = - new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + protected PulsarTestEnvironment pulsar = new PulsarTestEnvironment(runtime()); protected StreamExecutionEnvironment env; protected StreamTableEnvironment tableEnv; protected PulsarRuntime runtime() { - return PulsarRuntime.container(); + return PulsarRuntime.mock(); } - private static final int DEFAULT_PARALLELISM = 2; - @BeforeAll - public void beforeAll() { + void beforeAll() { pulsar.startUp(); - // run env + } + + @BeforeEach + void beforeEach() { env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(DEFAULT_PARALLELISM); env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); From 7bb30d9b3492ff9d6d908243af6b910604b20e7a Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Mon, 17 Oct 2022 16:06:15 +0800 Subject: [PATCH 19/20] 3: Stop using MiniClusterExtension as it will cause resource failure --- .../f4d91193-72ba-4ce4-ad83-98f780dce581 | 6 ++++++ .../flink-connector-pulsar/pom.xml | 8 +++++++ .../pulsar/table/PulsarTableITCase.java | 2 -- .../pulsar/table/PulsarTableTestBase.java | 21 +++++-------------- 4 files changed, 19 insertions(+), 18 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 b/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 index 40e7dc9686cf1..181e0a00b78c0 100644 --- a/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 +++ b/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 @@ -14,5 +14,11 @@ org.apache.flink.connector.pulsar.source.PulsarUnorderedSourceITCase does not sa * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.pulsar.table.PulsarTableITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index cac2407b74ea5..7100aa6ce0da6 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -311,6 +311,14 @@ under the License. pom import + + + net.java.dev.jna + jna + 5.5.0 + diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java index 67e076bbb7233..4ee36a6876167 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -31,7 +31,6 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -334,7 +333,6 @@ void sendMessageWithPropertiesAndReadPropertiesMetadata() throws Exception { @ParameterizedTest @MethodSource("provideSchemaData") - @Disabled() void readAndSelectIntoTableUsingSimpleSchema( String format, Schema schema, T value, String flinkTableDataType) throws Exception { final String sourceTopic = "source_topic_" + randomAlphanumeric(3); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java index 3140d48f267e7..1cd0a3669fa08 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -21,17 +21,14 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,24 +39,16 @@ public abstract class PulsarTableTestBase { private static final int DEFAULT_PARALLELISM = 1; - @RegisterExtension - protected static final MiniClusterExtension CLUSTER_EXTENSION = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); + protected MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); - protected PulsarTestEnvironment pulsar = new PulsarTestEnvironment(runtime()); + protected static PulsarTestEnvironment pulsar = new PulsarTestEnvironment(runtime()); protected StreamExecutionEnvironment env; protected StreamTableEnvironment tableEnv; - protected PulsarRuntime runtime() { - return PulsarRuntime.mock(); + protected static PulsarRuntime runtime() { + return PulsarRuntime.container(); } @BeforeAll From 614d9259688bd8512ebd478ef487359a67343694 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 18 Oct 2022 12:18:25 +0800 Subject: [PATCH 20/20] 4. remove unnecessary public declarations in test --- .../connector/pulsar/table/PulsarTableFactoryTest.java | 10 +++++----- .../connector/pulsar/table/PulsarTableTestBase.java | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java index 145571a918bd2..eb637bb7ec654 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java @@ -148,7 +148,7 @@ public class PulsarTableFactoryTest { private static final DataType SCHEMA_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); @Test - public void testTableSource() { + void testTableSource() { final Map configuration = getBasicSourceOptions(); final PulsarTableSource actualPulsarSource = (PulsarTableSource) createTableSource(SCHEMA, configuration); @@ -183,7 +183,7 @@ public void testTableSource() { } @Test - public void testTableSourceWithKeyValue() { + void testTableSourceWithKeyValue() { final Map configuration = getSourceKeyValueOptions(); final PulsarTableSource actualPulsarSource = @@ -228,7 +228,7 @@ public void testTableSourceWithKeyValue() { } @Test - public void testTableSourceWithKeyValueAndMetadata() { + void testTableSourceWithKeyValueAndMetadata() { final Map options = getSourceKeyValueOptions(); options.put("test-format.readable-metadata", "metadata_1:INT, metadata_2:STRING"); @@ -289,7 +289,7 @@ public void testTableSourceWithKeyValueAndMetadata() { } @Test - public void testTableSink() { + void testTableSink() { final Map modifiedOptions = getBasicSinkOptions(); final DynamicTableSink actualPulsarTableSink = createTableSink(SCHEMA, modifiedOptions); @@ -326,7 +326,7 @@ public void testTableSink() { } @Test - public void testTableSinkWithKeyValue() { + void testTableSinkWithKeyValue() { final Map modifiedOptions = getSinkKeyValueOptions(); final PulsarTableSink actualPulsarTableSink = (PulsarTableSink) createTableSink(SCHEMA, modifiedOptions); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java index 1cd0a3669fa08..b5a7779e9c486 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -72,7 +72,7 @@ public void createTestTopic(String topic, int numPartitions) { } @AfterAll - public void afterAll() { + void afterAll() { pulsar.tearDown(); } }