forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-18215: KIP-891 Connect Multiversioning Support (Configs and Val…
…idation changes for Connectors and Converters) (apache#17741) Reviewers: Greg Harris <[email protected]>
- Loading branch information
1 parent
5b58a3f
commit fd6ccdb
Showing
20 changed files
with
916 additions
and
266 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
409 changes: 246 additions & 163 deletions
409
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
Large diffs are not rendered by default.
Oops, something went wrong.
79 changes: 79 additions & 0 deletions
79
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/CachedConnectors.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,79 @@ | ||
/* | ||
* 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.kafka.connect.runtime; | ||
|
||
import org.apache.kafka.connect.connector.Connector; | ||
import org.apache.kafka.connect.runtime.isolation.Plugins; | ||
import org.apache.kafka.connect.runtime.isolation.VersionedPluginLoadingException; | ||
|
||
import org.apache.maven.artifact.versioning.VersionRange; | ||
|
||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
|
||
public class CachedConnectors { | ||
|
||
private static final String LATEST_VERSION = "latest"; | ||
|
||
private final Map<String, Map<String, Connector>> connectors; | ||
private final Map<String, Exception> invalidConnectors; | ||
private final Map<String, Map<String, Exception>> invalidVersions; | ||
private final Plugins plugins; | ||
|
||
public CachedConnectors(Plugins plugins) { | ||
this.plugins = plugins; | ||
this.connectors = new ConcurrentHashMap<>(); | ||
this.invalidConnectors = new ConcurrentHashMap<>(); | ||
this.invalidVersions = new ConcurrentHashMap<>(); | ||
} | ||
|
||
private void validate(String connectorName, VersionRange range) throws Exception { | ||
if (invalidConnectors.containsKey(connectorName)) { | ||
throw new Exception(invalidConnectors.get(connectorName)); | ||
} | ||
|
||
String version = range == null ? LATEST_VERSION : range.toString(); | ||
if (invalidVersions.containsKey(connectorName) && invalidVersions.get(connectorName).containsKey(version)) { | ||
throw new Exception(invalidVersions.get(connectorName).get(version)); | ||
} | ||
} | ||
|
||
private Connector lookup(String connectorName, VersionRange range) throws Exception { | ||
String version = range == null ? LATEST_VERSION : range.toString(); | ||
if (connectors.containsKey(connectorName) && connectors.get(connectorName).containsKey(version)) { | ||
return connectors.get(connectorName).get(version); | ||
} | ||
|
||
try { | ||
Connector connector = plugins.newConnector(connectorName, range); | ||
connectors.computeIfAbsent(connectorName, k -> new ConcurrentHashMap<>()).put(version, connector); | ||
return connector; | ||
} catch (VersionedPluginLoadingException e) { | ||
invalidVersions.computeIfAbsent(connectorName, k -> new ConcurrentHashMap<>()).put(version, e); | ||
throw e; | ||
} catch (Exception e) { | ||
invalidConnectors.put(connectorName, e); | ||
throw e; | ||
} | ||
} | ||
|
||
public Connector getConnector(String connectorName, VersionRange range) throws Exception { | ||
validate(connectorName, range); | ||
return lookup(connectorName, range); | ||
} | ||
} |
238 changes: 204 additions & 34 deletions
238
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
Large diffs are not rendered by default.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.