-
Notifications
You must be signed in to change notification settings - Fork 3.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[improve][fn] Allow unknown fields in connectors config #20116
Changes from 4 commits
8b68bb4
6ceaf22
fdcd65c
1650270
df2b82c
4cd1031
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,15 +24,18 @@ | |
import com.scurrilous.circe.checksum.Crc32cIntChecksum; | ||
import io.netty.buffer.ByteBuf; | ||
import java.io.IOException; | ||
import java.lang.reflect.Field; | ||
import java.nio.ByteBuffer; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Objects; | ||
import java.util.TreeMap; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
import java.util.concurrent.locks.ReadWriteLock; | ||
import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
import java.util.function.Consumer; | ||
import java.util.stream.Collectors; | ||
import lombok.Getter; | ||
import lombok.extern.slf4j.Slf4j; | ||
import net.jodah.typetools.TypeResolver; | ||
|
@@ -59,6 +62,7 @@ | |
import org.apache.pulsar.common.functions.ConsumerConfig; | ||
import org.apache.pulsar.common.functions.FunctionConfig; | ||
import org.apache.pulsar.common.functions.ProducerConfig; | ||
import org.apache.pulsar.common.nar.NarClassLoader; | ||
import org.apache.pulsar.common.protocol.schema.SchemaVersion; | ||
import org.apache.pulsar.common.schema.KeyValue; | ||
import org.apache.pulsar.common.schema.KeyValueEncodingType; | ||
|
@@ -94,6 +98,7 @@ | |
import org.apache.pulsar.functions.source.batch.BatchSourceExecutor; | ||
import org.apache.pulsar.functions.utils.CryptoUtils; | ||
import org.apache.pulsar.functions.utils.FunctionCommon; | ||
import org.apache.pulsar.functions.utils.io.ConnectorUtils; | ||
import org.apache.pulsar.io.core.Sink; | ||
import org.apache.pulsar.io.core.Source; | ||
import org.slf4j.Logger; | ||
|
@@ -855,10 +860,7 @@ private void setupInput(ContextImpl contextImpl) throws Exception { | |
if (sourceSpec.getConfigs().isEmpty()) { | ||
this.source.open(new HashMap<>(), contextImpl); | ||
} else { | ||
this.source.open( | ||
ObjectMapperFactory.getMapper().reader().forType(new TypeReference<Map<String, Object>>() { | ||
}).readValue(sourceSpec.getConfigs()) | ||
, contextImpl); | ||
this.source.open(parseComponentConfig(sourceSpec.getConfigs()), contextImpl); | ||
} | ||
if (this.source instanceof PulsarSource) { | ||
contextImpl.setInputConsumers(((PulsarSource) this.source).getInputConsumers()); | ||
|
@@ -870,6 +872,56 @@ private void setupInput(ContextImpl contextImpl) throws Exception { | |
Thread.currentThread().setContextClassLoader(this.instanceClassLoader); | ||
} | ||
} | ||
private Map<String, Object> parseComponentConfig(String connectorConfigs) throws IOException { | ||
return parseComponentConfig(connectorConfigs, instanceConfig, componentClassLoader, componentType); | ||
} | ||
|
||
static Map<String, Object> parseComponentConfig(String connectorConfigs, | ||
InstanceConfig instanceConfig, | ||
ClassLoader componentClassLoader, | ||
org.apache.pulsar.functions.proto.Function | ||
.FunctionDetails.ComponentType componentType) | ||
throws IOException { | ||
final Map<String, Object> config = ObjectMapperFactory | ||
.getMapper() | ||
.reader() | ||
.forType(new TypeReference<Map<String, Object>>() {}) | ||
.readValue(connectorConfigs); | ||
if (instanceConfig.isIgnoreUnknownConfigFields() && componentClassLoader instanceof NarClassLoader) { | ||
final String configClass; | ||
if (componentType == org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType.SOURCE) { | ||
configClass = ConnectorUtils | ||
.getConnectorDefinition((NarClassLoader) componentClassLoader).getSourceConfigClass(); | ||
} else if (componentType == org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType.SINK) { | ||
configClass = ConnectorUtils | ||
.getConnectorDefinition((NarClassLoader) componentClassLoader).getSinkConfigClass(); | ||
} else { | ||
return config; | ||
} | ||
if (configClass != null) { | ||
final Object configInstance = Reflections.createInstance(configClass, | ||
Thread.currentThread().getContextClassLoader()); | ||
final List<String> allFields = | ||
Reflections | ||
.getAllFields(configInstance.getClass()) | ||
.stream() | ||
.map(Field::getName) | ||
.collect(Collectors.toList()); | ||
|
||
for (String s : config.keySet()) { | ||
if (!allFields.contains(s)) { | ||
log.warn("Field '{}' not defined in the {} configuration {}, the field will be ignored", | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. maybe this should be logged as "ERROR", WARNINGs tend to be ignored by alert systems There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
s, | ||
componentType, | ||
configClass); | ||
config.remove(s); | ||
} | ||
} | ||
} | ||
} | ||
nicoloboschi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
return config; | ||
} | ||
|
||
|
||
private void setupOutput(ContextImpl contextImpl) throws Exception { | ||
|
||
|
@@ -940,9 +992,8 @@ private void setupOutput(ContextImpl contextImpl) throws Exception { | |
log.debug("Opening Sink with SinkSpec {} and contextImpl: {} ", sinkSpec, | ||
contextImpl.toString()); | ||
} | ||
this.sink.open(ObjectMapperFactory.getMapper().reader().forType( | ||
new TypeReference<Map<String, Object>>() { | ||
}).readValue(sinkSpec.getConfigs()), contextImpl); | ||
final Map<String, Object> config = parseComponentConfig(sinkSpec.getConfigs()); | ||
this.sink.open(config, contextImpl); | ||
} | ||
} catch (Exception e) { | ||
log.error("Sink open produced uncaught exception: ", e); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -738,6 +738,17 @@ public String getFunctionAuthProviderClassName() { | |
) | ||
private List<String> additionalJavaRuntimeArguments = new ArrayList<>(); | ||
|
||
@FieldContext( | ||
category = CATEGORY_CONNECTORS, | ||
doc = "Whether to ignore unknown properties when deserializing the connector configuration. " | ||
+ "After upgrading a connector to a new version with a new configuration, " | ||
+ "the new configuration may not be compatible with the old connector. " | ||
+ "In case of rollback, it's required to also rollback the connector configuration. " | ||
+ "Ignoring unknown fields makes possible to keep the new configuration and " | ||
+ "only rollback the connector." | ||
) | ||
private boolean ignoreUnknownConfigFields = false; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This configuration applies to the instances of the functions/connectors, and not to the function worker (that already ignores unknown fields) what about 'functionsIgnoreUnknownConfigFields' ? (maybe we can do better, but connectors are actually functions) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I thought about that and I agree. But there are other properties that are meant to be used by the java runner like |
||
|
||
public String getFunctionMetadataTopic() { | ||
return String.format("persistent://%s/%s", pulsarFunctionsNamespace, functionMetadataTopicName); | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am not sure that this is correct.
ObjectMapper should follow Java Beans conventions and use getter/setters together with public fields.
We should use some ObjectMapper utilities here in order to ensure that we are doing the right thing
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I've found a way to use the same deserializer used by jackson mapper, so this should cover all the cases when jackson mapper is used by the sink/source