Unverified Commit 5fe68097 authored by DarrenDa's avatar DarrenDa Committed by GitHub

add dlink-connector-pulsar (#723)

* add dlink-connector-pulsar

* replace json resolve
Co-authored-by: 's avatarfada.yu <fada.yu@zhaopin.com.cn>
parent 04937cb6
This diff is collapsed.
This diff is collapsed.
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.dlink.connector.pulsar;
import com.sun.istack.internal.Nullable;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.sink.SinkFunctionProvider;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.flink.types.RowKind;
import java.util.Collections;
import java.util.List;
import java.util.Properties;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* @author DarrenDa
* * @version 1.0
* * @Desc:
**/
/**
* A version-agnostic Pulsar {@link DynamicTableSink}.
*/
@Internal
public class PulsarDynamicSink implements DynamicTableSink {
// --------------------------------------------------------------------------------------------
// Mutable attributes
// --------------------------------------------------------------------------------------------
/**
* Metadata that is appended at the end of a physical sink row.
*/
protected List<String> metadataKeys;
// --------------------------------------------------------------------------------------------
// Format attributes
// --------------------------------------------------------------------------------------------
/**
* Data type of consumed data type.
*/
protected DataType consumedDataType;
/**
* Data type to configure the formats.
*/
protected final DataType physicalDataType;
/**
* Optional format for encoding to Pulsar.
*/
protected final @Nullable
EncodingFormat<SerializationSchema<RowData>> encodingFormat;
// --------------------------------------------------------------------------------------------
// Pulsar-specific attributes
// --------------------------------------------------------------------------------------------
/**
* The Pulsar topic to write to.
*/
protected final String topic;
/**
* The Pulsar service url config.
*/
protected final String serviceUrl;
/**
* The Pulsar update mode to.
*/
protected final String updateMode;
/**
* Properties for the Pulsar producer.
*/
protected final Properties pulsarProducerProperties;
/**
* Properties for the Pulsar producer.
*/
protected final Properties pulsarClientProperties;
/**
* Properties for the Pulsar producer parallelism.
*/
protected final Integer sinkParallelism;
public PulsarDynamicSink(
DataType physicalDataType,
@Nullable EncodingFormat<SerializationSchema<RowData>> encodingFormat,
String topic,
String service_url,
String update_mode,
Properties pulsarProducerProperties,
Properties pulsarClientProperties,
Integer sinkParallelism) {
// Format attributes
this.physicalDataType =
checkNotNull(physicalDataType, "Physical data type must not be null.");
this.encodingFormat = encodingFormat;
// Mutable attributes
this.metadataKeys = Collections.emptyList();
// Pulsar-specific attributes
this.topic = checkNotNull(topic, "Topic must not be null.");
this.serviceUrl = checkNotNull(service_url, "Service url must not be null.");
this.updateMode = checkNotNull(update_mode, "Update mode must not be null.");
this.pulsarProducerProperties = checkNotNull(pulsarProducerProperties, "pulsarProducerProperties must not be null.");
this.pulsarClientProperties = checkNotNull(pulsarClientProperties, "pulsarClientProperties must not be null.");
this.sinkParallelism = sinkParallelism;
}
@Override
public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
if (updateMode.equals("append")) {
return ChangelogMode.newBuilder()
.addContainedKind(RowKind.INSERT)
.build();
} else {
return ChangelogMode.newBuilder()
.addContainedKind(RowKind.INSERT)
// .addContainedKind(RowKind.UPDATE_BEFORE)
// .addContainedKind(RowKind.DELETE)
.addContainedKind(RowKind.UPDATE_AFTER)
.build();
}
// return encodingFormat.getChangelogMode();
}
@Override
public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
SerializationSchema<RowData> runtimeEncoder = encodingFormat.createRuntimeEncoder(context, physicalDataType);
PulsarSinkFunction<RowData> sinkFunction =
new PulsarSinkFunction<>(
topic,
serviceUrl,
pulsarProducerProperties,
pulsarClientProperties,
runtimeEncoder);
//sink的并行度设置
if (sinkParallelism != null) {
return SinkFunctionProvider.of(sinkFunction, sinkParallelism);
} else {
return SinkFunctionProvider.of(sinkFunction);
}
}
@Override
public DynamicTableSink copy() {
final PulsarDynamicSink copy =
new PulsarDynamicSink(
physicalDataType,
encodingFormat,
topic,
serviceUrl,
updateMode,
pulsarProducerProperties,
pulsarClientProperties,
sinkParallelism);
copy.metadataKeys = metadataKeys;
return copy;
}
@Override
public String asSummaryString() {
return "Pulsar table sink";
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.dlink.connector.pulsar;
import com.dlink.connector.pulsar.util.PulsarConnectorOptions;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.pulsar.source.PulsarSource;
import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder;
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.reader.deserializer.PulsarDeserializationSchema;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.pulsar.client.api.SubscriptionType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.Properties;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* @author DarrenDa
* * @version 1.0
* * @Desc:
**/
/** A version-agnostic Pulsar {@link ScanTableSource}. */
@Internal
public class PulsarDynamicSource
implements ScanTableSource, SupportsWatermarkPushDown {
private static final Logger LOG = LoggerFactory.getLogger(PulsarDynamicSource.class);
private final String serviceUrl;
private final String adminUrl;
private final String subscriptionName;
private final SubscriptionType subscriptionType;
private final PulsarConnectorOptions.ScanStartupMode startupMode;
private final String topic;
private final DecodingFormat<DeserializationSchema<RowData>> decodingFormat;
private final DataType producedDataType;
private final String tableIdentifier;
private final Properties properties;
private final Long timestamp;
private final Integer sourceParallelism;
/** Watermark strategy that is used to generate per-partition watermark. */
protected WatermarkStrategy<RowData> watermarkStrategy;
public PulsarDynamicSource(
String serviceUrl,
String adminUrl,
String subscriptionName,
SubscriptionType subscriptionType,
PulsarConnectorOptions.ScanStartupMode startupMode,
Long timestamp,
String topic,
DecodingFormat<DeserializationSchema<RowData>> decodingFormat,
DataType producedDataType,
String tableIdentifier,
Properties properties,
Integer sourceParallelism) {
this.serviceUrl = serviceUrl;
this.adminUrl = adminUrl;
this.subscriptionName = subscriptionName;
this.subscriptionType = subscriptionType;
this.startupMode = startupMode;
this.timestamp = timestamp;
this.topic = topic;
this.decodingFormat = decodingFormat;
this.producedDataType = producedDataType;
this.tableIdentifier = tableIdentifier;
this.properties = properties;
this.sourceParallelism = sourceParallelism;
this.watermarkStrategy = null;
}
@Override
public ChangelogMode getChangelogMode() {
// in our example the format decides about the changelog mode
// but it could also be the source itself
return decodingFormat.getChangelogMode();
}
@Override
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
// create runtime classes that are shipped to the cluster
final DeserializationSchema<RowData> deserializer =
decodingFormat.createRuntimeDecoder(runtimeProviderContext, producedDataType);
final PulsarSource<RowData> pulsarSource =
createPulsarSource(deserializer);
return new DataStreamScanProvider() {
@Override
public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv) {
if (watermarkStrategy == null) {
LOG.info("WatermarkStrategy 为空");
watermarkStrategy = WatermarkStrategy.noWatermarks();
} else {
LOG.info("WatermarkStrategy 不为空");
}
DataStreamSource<RowData> rowDataDataStreamSource = execEnv.fromSource(
pulsarSource, watermarkStrategy, "PulsarSource-" + tableIdentifier);
//设置source并行度
if (sourceParallelism != null) {
rowDataDataStreamSource.setParallelism(sourceParallelism);
}
return rowDataDataStreamSource;
}
@Override
public boolean isBounded() {
return pulsarSource.getBoundedness() == Boundedness.BOUNDED;
}
};
}
@Override
public DynamicTableSource copy() {
return new PulsarDynamicSource(
serviceUrl,
adminUrl,
subscriptionName,
subscriptionType,
startupMode,
timestamp,
topic,
decodingFormat,
producedDataType,
tableIdentifier,
properties,
sourceParallelism
);
}
@Override
public String asSummaryString() {
return "Pulsar Table Source";
}
//---------------------------------------------------------------------------------------------
protected PulsarSource<RowData> createPulsarSource(
DeserializationSchema<RowData> deserializer) {
final PulsarSourceBuilder<RowData> pulsarSourceBuilder = PulsarSource.builder();
pulsarSourceBuilder
.setConfig(PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE, true)
.setServiceUrl(serviceUrl)
.setAdminUrl(adminUrl)
.setTopics(topic)
.setDeserializationSchema(PulsarDeserializationSchema.flinkSchema(deserializer))
.setConfig(Configuration.fromMap((Map) properties))
.setSubscriptionName(subscriptionName);
switch (subscriptionType) {
case Shared:
pulsarSourceBuilder.setSubscriptionType(SubscriptionType.Shared);
break;
case Exclusive:
pulsarSourceBuilder.setSubscriptionType(SubscriptionType.Exclusive);
break;
case Key_Shared:
pulsarSourceBuilder.setSubscriptionType(SubscriptionType.Key_Shared);
break;
case Failover:
pulsarSourceBuilder.setSubscriptionType(SubscriptionType.Failover);
break;
default:
throw new TableException(
"Unsupported subscriptionType. Validator should have checked that.");
}
switch (startupMode) {
case EARLIEST:
pulsarSourceBuilder.setStartCursor(StartCursor.earliest());
break;
case LATEST:
pulsarSourceBuilder.setStartCursor(StartCursor.latest());
break;
case TIMESTAMP:
checkNotNull(timestamp, "No timestamp supplied.");
pulsarSourceBuilder.setStartCursor(StartCursor.fromMessageTime(timestamp));
break;
default:
throw new TableException(
"Unsupported startup mode. Validator should have checked that.");
}
return pulsarSourceBuilder.build();
}
@Override
public void applyWatermark(WatermarkStrategy<RowData> watermarkStrategy) {
this.watermarkStrategy = watermarkStrategy;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.dlink.connector.pulsar;
import org.apache.flink.annotation.Internal;
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.table.api.ValidationException;
import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.catalog.ObjectIdentifier;
import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.connector.format.Format;
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.*;
import org.apache.flink.table.factories.FactoryUtil.TableFactoryHelper;
import org.apache.flink.table.types.DataType;
import org.apache.pulsar.client.api.SubscriptionType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.util.HashSet;
import java.util.Properties;
import java.util.Set;
import static com.dlink.connector.pulsar.util.PulsarConnectorOptions.*;
import static com.dlink.connector.pulsar.util.PulsarConnectorOptionsUtil.*;
/**
* Factory for creating configured instances of {@link PulsarDynamicSource} and {
*
* @author DarrenDa
* * @version 1.0
* * @Desc:
* @link PulsarDynamicSink}.
*/
@Internal
public class PulsarDynamicTableFactory
implements DynamicTableSourceFactory, DynamicTableSinkFactory {
private static final Logger LOG = LoggerFactory.getLogger(PulsarDynamicTableFactory.class);
public static final String IDENTIFIER = "pulsar";
@Override
public String factoryIdentifier() {
return IDENTIFIER;
}
@Override
public Set<ConfigOption<?>> requiredOptions() {
final Set<ConfigOption<?>> options = new HashSet<>();
options.add(SERVICE_URL);
return options;
}
@Override
public Set<ConfigOption<?>> optionalOptions() {
final Set<ConfigOption<?>> options = new HashSet<>();
options.add(ADMIN_URL);
options.add(SUBSCRIPTION_NAME);
options.add(SUBSCRIPTION_TYPE);
options.add(SUBSCRIPTION_INITIAL_POSITION);
options.add(SUBSCRIPTION_INITIAL_POSITION_TIMESTAMP);
options.add(FactoryUtil.FORMAT);
options.add(TOPIC);
options.add(UPDATE_MODE);
options.add(SOURCE_PARALLELISM);
options.add(SINK_PARALLELISM);
options.add(VERSION);
options.add(DERIVE_SCHEMA);
return options;
}
@Override
public DynamicTableSource createDynamicTableSource(Context context) {
// either implement your custom validation logic here ...
// or use the provided helper utility
final TableFactoryHelper helper =
FactoryUtil.createTableFactoryHelper(this, context);
// discover a suitable decoding format
final DecodingFormat<DeserializationSchema<RowData>> decodingFormat =
helper.discoverDecodingFormat(
DeserializationFormatFactory.class, FactoryUtil.FORMAT);
// validate all options
// helper.validate();
helper.validateExcept(PROPERTIES_PREFIX, PROPERTIES_CLIENT_PREFIX);
// get the validated options
final ReadableConfig tableOptions = helper.getOptions();
final String serviceUrl = tableOptions.get(SERVICE_URL);
final String adminUrl = tableOptions.get(ADMIN_URL);
final String subscriptionName = tableOptions.get(SUBSCRIPTION_NAME);
final SubscriptionType subscriptionType = tableOptions.get(SUBSCRIPTION_TYPE);
final ScanStartupMode startupMode = tableOptions.get(SUBSCRIPTION_INITIAL_POSITION);
final Long timestamp = tableOptions.get(SUBSCRIPTION_INITIAL_POSITION_TIMESTAMP);
final String topic = tableOptions.get(TOPIC);
final Integer sourceParallelism = tableOptions.get(SOURCE_PARALLELISM);
// derive the produced data type (excluding computed columns) from the catalog table
final DataType producedDataType =
context.getCatalogTable().getResolvedSchema().toPhysicalRowDataType();
// create and return dynamic table source
return new PulsarDynamicSource(
serviceUrl,
adminUrl,
subscriptionName,
subscriptionType,
startupMode,
timestamp,
topic,
decodingFormat,
producedDataType,
context.getObjectIdentifier().asSummaryString(),
getPulsarProperties(context.getCatalogTable().getOptions(), PROPERTIES_PREFIX),
sourceParallelism
);
}
@Override
public DynamicTableSink createDynamicTableSink(Context context) {
final TableFactoryHelper helper =
FactoryUtil.createTableFactoryHelper(
this, context);
final ReadableConfig tableOptions = helper.getOptions();
final String update_mode = tableOptions.get(UPDATE_MODE);
final Integer sinkParallelism = tableOptions.get(SINK_PARALLELISM);
helper.validateExcept(PROPERTIES_PREFIX, PROPERTIES_CLIENT_PREFIX);
final EncodingFormat<SerializationSchema<RowData>> encodingFormat =
helper.discoverEncodingFormat(
SerializationFormatFactory.class, FactoryUtil.FORMAT);
//校验sql建表时是否指定主键约束
//我们一般使用flink自动推导出来的主键,不显式设置主键约束,所以这个校验方法暂时不使用
// validatePKConstraints(
// update_mode, context.getObjectIdentifier(), context.getCatalogTable(), encodingFormat);
final DataType physicalDataType =
context.getCatalogTable().getSchema().toPhysicalRowDataType();
return createPulsarTableSink(
physicalDataType,
encodingFormat,
tableOptions.get(TOPIC),
tableOptions.get(SERVICE_URL),
update_mode,
getPulsarProperties(context.getCatalogTable().getOptions(), PROPERTIES_PREFIX),
getPulsarProperties(context.getCatalogTable().getOptions(), PROPERTIES_CLIENT_PREFIX),
sinkParallelism
);
}
//校验sql建表时是否指定主键约束
private static void validatePKConstraints(
@Nullable String update_mode, ObjectIdentifier tableName, CatalogTable catalogTable, Format format) {
if (!update_mode.equals("append") && !update_mode.equals("upsert")) {
throw new ValidationException(
String.format(
"The Pulsar table '%s' with update-mode should be 'append' or 'upsert'",
tableName.asSummaryString()));
} else if (catalogTable.getSchema().getPrimaryKey().isPresent()
&& update_mode.equals("append")) {
throw new ValidationException(
String.format(
"The Pulsar table '%s' with append update-mode doesn't support defining PRIMARY KEY constraint"
+ " on the table, because it can't guarantee the semantic of primary key.",
tableName.asSummaryString()));
} else if (!catalogTable.getSchema().getPrimaryKey().isPresent()
&& update_mode.equals("upsert")) {
throw new ValidationException(
"'upsert' tables 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' table should update or delete on which keys.");
}
}
protected PulsarDynamicSink createPulsarTableSink(
DataType physicalDataType,
@Nullable EncodingFormat<SerializationSchema<RowData>> encodingFormat,
String topic,
String service_url,
String update_mode,
Properties pulsarProducerProperties,
Properties pulsarClientProperties,
Integer sinkParallelism) {
return new PulsarDynamicSink(
physicalDataType,
encodingFormat,
topic,
service_url,
update_mode,
pulsarProducerProperties,
pulsarClientProperties,
sinkParallelism);
}
}
package com.dlink.connector.pulsar.util;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.Properties;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
/**
* @author DarrenDa
* * @version 1.0
* * @Desc:
*/
public class PulsarConnectionHolder {
private static final Logger LOG = LoggerFactory.getLogger(PulsarConnectionHolder.class);
private static final Map<String, PulsarClientImpl> PULSAR_CLIENT_MAP = new ConcurrentHashMap<>();
public static PulsarClientImpl getConsumerClient(String serviceUrl, Properties properties) throws Exception {
return get(serviceUrl, true, properties);
}
public static PulsarClientImpl getProducerClient(String serviceUrl, Properties properties) throws Exception {
return get(serviceUrl, false, properties);
}
private static PulsarClientImpl get(String serviceUrl, boolean consumer, Properties properties) throws Exception {
synchronized (PulsarConnectionHolder.class) {
String pulsarClientCacheKey = getPulsarClientCacheKey(serviceUrl, consumer);
PulsarClientImpl pulsarClient = PULSAR_CLIENT_MAP.get(pulsarClientCacheKey);
if (null != pulsarClient) {
return pulsarClient;
}
// return PULSAR_CLIENT_MAP.computeIfAbsent(pulsarClientCacheKey, serviceUrlTag -> createPulsarClient(serviceUrl));
PulsarClientImpl pulsarClientImpl = createPulsarClient(serviceUrl, properties);
PulsarClientImpl newPulsarClientImpl = PULSAR_CLIENT_MAP.putIfAbsent(pulsarClientCacheKey, pulsarClientImpl);
if (newPulsarClientImpl == null) {
return pulsarClientImpl;
}
return newPulsarClientImpl;
}
}
private static String getPulsarClientCacheKey(String serviceUrl, boolean consumer) {
return serviceUrl + consumer;
}
private static PulsarClientImpl createPulsarClient(String serviceUrl, Properties properties) {
try {
LOG.info("create client, and ID is " + UUID.randomUUID() + ", and cache map size is " + PULSAR_CLIENT_MAP.size());
return (PulsarClientImpl) PulsarClient
.builder()
.serviceUrl(serviceUrl)
.maxNumberOfRejectedRequestPerConnection(50)
.loadConf((Map) properties)
.build();
} catch (Exception e) {
e.printStackTrace();
throw new RuntimeException("创建PulsarClient失败", e);
}
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.dlink.connector.pulsar.util;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.DescribedEnum;
import org.apache.flink.configuration.description.InlineElement;
import org.apache.pulsar.client.api.SubscriptionType;
import static org.apache.flink.configuration.description.TextElement.text;
/**
* @author DarrenDa
* * @version 1.0
* * @Desc:
**/
/** Options for the Pulsar connector. */
@PublicEvolving
public class PulsarConnectorOptions {
// --------------------------------------------------------------------------------------------
// Format options
// --------------------------------------------------------------------------------------------
public static final ConfigOption<String> SERVICE_URL =
ConfigOptions.key("connector.service-url")
.stringType()
.noDefaultValue()
.withDescription(
"Defines pulsar service url. ");
public static final ConfigOption<String> ADMIN_URL =
ConfigOptions.key("connector.admin-url")
.stringType()
.defaultValue("http://pulsar-dlink-qa.dlink.com:8080")
.withDescription(
"Defines pulsar admin url. ");
public static final ConfigOption<String> TOPIC =
ConfigOptions.key("connector.topic")
.stringType()
.noDefaultValue()
.withDescription(
"Defines pulsar topic. ");
public static final ConfigOption<String> SUBSCRIPTION_NAME =
ConfigOptions.key("connector.subscription-name")
.stringType()
.noDefaultValue()
.withDescription(
"Defines pulsar subscription name. ");
public static final ConfigOption<SubscriptionType> SUBSCRIPTION_TYPE =
ConfigOptions.key("connector.subscription-type")
.enumType(SubscriptionType.class)
.defaultValue(SubscriptionType.Shared)
.withDescription(
"Defines pulsar subscription type. ");
public static final ConfigOption<ScanStartupMode> SUBSCRIPTION_INITIAL_POSITION =
ConfigOptions.key("connector.subscription-initial-position")
.enumType(ScanStartupMode.class)
.defaultValue(ScanStartupMode.LATEST)
.withDescription("Startup mode for Pulsar consumer.");
public static final ConfigOption<Long> SUBSCRIPTION_INITIAL_POSITION_TIMESTAMP =
ConfigOptions.key("connector.subscription-initial-position.timestamp")
.longType()
.noDefaultValue()
.withDescription("Start from the specified message time by Message<byte[]>.getPublishTime().");
public static final ConfigOption<String> UPDATE_MODE =
ConfigOptions.key("update-mode")
.stringType()
.noDefaultValue()
.withDescription(
"Defines pulsar update mode. ");
public static final ConfigOption<Integer> SOURCE_PARALLELISM =
ConfigOptions.key("source-parallelism")
.intType()
.noDefaultValue()
.withDescription(
"Defines pulsar sink parallelism. ");
public static final ConfigOption<Integer> SINK_PARALLELISM =
ConfigOptions.key("sink-parallelism")
.intType()
.noDefaultValue()
.withDescription(
"Defines pulsar sink parallelism. ");
//与老平台 1.14.3之前版本的sql进行兼容,但是并未使用的参数
public static final ConfigOption<String> VERSION =
ConfigOptions.key("connector.version")
.stringType()
.noDefaultValue()
.withDescription(
"Defines pulsar version. ");
//与老平台 1.14.3之前版本的sql进行兼容,但是并未使用的参数
public static final ConfigOption<String> DERIVE_SCHEMA =
ConfigOptions.key("format.derive-schema")
.stringType()
.noDefaultValue()
.withDescription(
"Defines pulsar derive schema. ");
// public static final ConfigOption<Boolean> PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE =
// ConfigOptions.key("pulsar.source.enableAutoAcknowledgeMessage")
// .booleanType()
// .noDefaultValue()
// .withDescription(
// "Defines pulsar enable auto acknowledge message. ");
// --------------------------------------------------------------------------------------------
// Enums
// --------------------------------------------------------------------------------------------
/** Startup mode for the Pulsar consumer, see {@link #SUBSCRIPTION_INITIAL_POSITION}. */
public enum ScanStartupMode implements DescribedEnum {
EARLIEST("Earliest", text("Start from the earliest available message in the topic..")),
LATEST("Latest", text("Start from the latest available message in the topic.")),
TIMESTAMP("Timestamp", text("Start from the specified message time by Message<byte[]>.getPublishTime()."));
private final String value;
private final InlineElement description;
ScanStartupMode(String value, InlineElement description) {
this.value = value;
this.description = description;
}
@Override
public String toString() {
return value;
}
@Override
public InlineElement getDescription() {
return description;
}
}
private PulsarConnectorOptions() {
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.dlink.connector.pulsar.util;
import org.apache.flink.annotation.PublicEvolving;
import java.util.Map;
import java.util.Properties;
/**
* @author DarrenDa
* * @version 1.0
* * @Desc:
**/
/** Utilities for {@link PulsarConnectorOptions}. */
@PublicEvolving
public class PulsarConnectorOptionsUtil {
// Prefix for Pulsar specific properties.
public static final String PROPERTIES_PREFIX = "properties.";
public static final String PROPERTIES_CLIENT_PREFIX = "properties_client.";
public static Properties getPulsarProperties(Map<String, String> tableOptions, String prefix) {
final Properties pulsarProperties = new Properties();
if (hasPulsarClientProperties(tableOptions)) {
tableOptions.keySet().stream()
.filter(key -> key.startsWith(prefix))
.forEach(
key -> {
final String value = tableOptions.get(key);
final String subKey = key.substring((prefix).length());
pulsarProperties.put(subKey, value);
});
}
return pulsarProperties;
}
/**
* Decides if the table options contains Pulsar client properties that start with prefix
* 'properties'.
*/
private static boolean hasPulsarClientProperties(Map<String, String> tableOptions) {
return tableOptions.keySet().stream().anyMatch(k -> k.startsWith(PROPERTIES_PREFIX));
}
private PulsarConnectorOptionsUtil() {
}
}
package com.dlink.connector.pulsar.util;
import org.apache.pulsar.client.api.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.Properties;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
/**
* @author DarrenDa
* * @version 1.0
* * @Desc:
*/
public class PulsarProducerHolder {
private static final Logger LOG = LoggerFactory.getLogger(PulsarProducerHolder.class);
private static final Map<String, Producer> PULSAR_PRODUCER_MAP = new ConcurrentHashMap<>();
public static Producer getProducer(String defaultTopicName, Properties properties, PulsarClient client) throws Exception {
return get(defaultTopicName, properties, client);
}
private static Producer get(String defaultTopicName, Properties properties, PulsarClient client) throws Exception {
synchronized (PulsarProducerHolder.class) {
String pulsarProducerCacheKey = defaultTopicName;
Producer pulsarProducer = PULSAR_PRODUCER_MAP.get(pulsarProducerCacheKey);
LOG.info("get pulsarProducer from map result is " + pulsarProducer);
if (null != pulsarProducer) {
return pulsarProducer;
}
Producer producer = createPulsarProducer(defaultTopicName, properties, client);
Producer newPulsarProducer = PULSAR_PRODUCER_MAP.putIfAbsent(pulsarProducerCacheKey, producer);
if (newPulsarProducer == null) {
return producer;
}
return newPulsarProducer;
}
}
private static Producer createPulsarProducer(String defaultTopicName, Properties properties, PulsarClient client) {
try {
LOG.info("create producer, and ID is " + UUID.randomUUID() + ", and cache map size is " + PULSAR_PRODUCER_MAP.size());
LOG.info("now defaultTopicName is " + defaultTopicName + ", and map content is " + PULSAR_PRODUCER_MAP.get(defaultTopicName));
ProducerBuilder<byte[]> producerBuilder = client.newProducer();
producerBuilder.
blockIfQueueFull(Boolean.TRUE).
compressionType(CompressionType.LZ4).
topic(defaultTopicName).
hashingScheme(HashingScheme.JavaStringHash).
// batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS).
loadConf((Map) properties);
Producer<byte[]> producer = producerBuilder.create();
return producer;
// return client.newProducer().
// blockIfQueueFull(Boolean.TRUE).
// compressionType(CompressionType.LZ4).
// topic(defaultTopicName).
// hashingScheme(HashingScheme.JavaStringHash).
//// batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS).
// loadConf((Map)properties).
// create();
} catch (Exception e) {
e.printStackTrace();
throw new RuntimeException("创建Producer失败", e);
}
}
}
# 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.
com.dlink.connector.pulsar.PulsarDynamicTableFactory
package com.dlink.connector.pulsar;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.Table;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.TableResult;
import org.junit.Test;
/**
* @author DarrenDa
* @version 1.0
* @Desc: Test case
*/
public class PulsarSqlCase {
@Test
public void testCase() {
EnvironmentSettings settings = EnvironmentSettings.newInstance()
.inStreamingMode()
.build();
TableEnvironment tableEnvironment = TableEnvironment.create(settings);
tableEnvironment.executeSql("create table source_gen_data(\n" +
" f_sequence INT,\n" +
" f_random INT,\n" +
" f_random_str STRING,\n" +
" ts AS localtimestamp,\n" +
" WATERMARK FOR ts AS ts\n" +
") WITH (\n" +
" 'connector' = 'datagen',\n" +
"\n" +
" -- optional options --\n" +
"\n" +
" 'rows-per-second'='5',\n" +
"\n" +
" 'fields.f_sequence.kind'='sequence',\n" +
" 'fields.f_sequence.start'='1',\n" +
" 'fields.f_sequence.end'='1000',\n" +
"\n" +
" 'fields.f_random.min'='1',\n" +
" 'fields.f_random.max'='1000',\n" +
"\n" +
" 'fields.f_random_str.length'='10'\n" +
")")
;
tableEnvironment.executeSql("create table sink_table(\n" +
" f_sequence INT,\n" +
" f_random INT,\n" +
" f_random_str STRING,\n" +
" ts string\n" +
") with (\n" +
" 'connector' = 'print'\n" +
")");
TableResult tableResult = tableEnvironment.executeSql("insert into sink_table\n" +
"select\n" +
" f_sequence ,\n" +
" f_random ,\n" +
" f_random_str ,\n" +
" cast(ts as string)\n" +
"from source_gen_data");
tableResult.print();
// Table tb = tableEnvironment.sqlQuery("select * from source_gen_data");
// tableResult.execute().print();
}
@Test
public void pulsarTest() throws Exception {
EnvironmentSettings settings = EnvironmentSettings.newInstance()
.inStreamingMode()
.build();
TableEnvironment tableEnvironment = TableEnvironment.create(settings);
tableEnvironment.executeSql("" +
"CREATE TABLE source_pulsar(\n" +
" requestId VARCHAR,\n" +
" `timestamp` BIGINT,\n" +
" `date` VARCHAR,\n" +
" appId VARCHAR,\n" +
" appName VARCHAR,\n" +
" forwardTimeMs VARCHAR,\n" +
" processingTimeMs INT,\n" +
" errCode VARCHAR,\n" +
" userIp VARCHAR,\n" +
" createTime bigint,\n" +
" b_create_time as TO_TIMESTAMP(FROM_UNIXTIME(createTime/1000,'yyyy-MM-dd HH:mm:ss'),'yyyy-MM-dd HH:mm:ss')\n" +
") WITH (\n" +
" 'connector' = 'pulsar',\n" +
" 'connector.version' = 'universal',\n" +
" 'connector.topic' = 'persistent://dlink/dev/context.pulsar',\n" +
" 'connector.service-url' = 'pulsar://pulsar-dlink-n.stream.com:6650',\n" +
" 'connector.subscription-name' = 'tmp_print_detail',\n" +
" 'connector.subscription-type' = 'Shared',\n" +
" 'connector.subscription-initial-position' = 'Latest',\n" +
" 'update-mode' = 'append',\n" +
" 'format' = 'json',\n" +
" 'format.derive-schema' = 'true'\n" +
")")
;
tableEnvironment.executeSql("" +
"create table sink_pulsar_result(\n" +
" requestId VARCHAR,\n" +
" `timestamp` BIGINT,\n" +
" `date` VARCHAR,\n" +
" appId VARCHAR,\n" +
" appName VARCHAR,\n" +
" forwardTimeMs VARCHAR,\n" +
" processingTimeMs INT,\n" +
" errCode VARCHAR,\n" +
" userIp VARCHAR\n" +
") with (\n" +
" 'connector' = 'print'\n" +
")");
TableResult tableResult = tableEnvironment.executeSql("" +
"insert into sink_pulsar_result\n" +
"select \n" +
" requestId ,\n" +
" `timestamp`,\n" +
" `date`,\n" +
" appId,\n" +
" appName,\n" +
" forwardTimeMs,\n" +
" processingTimeMs,\n" +
" errCode,\n" +
" userIp\n" +
"from source_pulsar");
tableResult.print();
}
}
# 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.
com.dlink.connector.pulsar.PulsarDynamicTableFactory
......@@ -17,6 +17,7 @@
<module>dlink-connector-phoenix-1.13</module>
<module>dlink-connector-phoenix-1.14</module>
<module>dlink-connector-doris-1.13</module>
<module>dlink-connector-pulsar-1.14</module>
</modules>
<artifactId>dlink-connectors</artifactId>
</project>
\ No newline at end of file
</project>
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment