Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 5 additions & 1 deletion bin/kafka-run-class.sh
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,11 @@ fi
shopt -s nullglob
for dir in $base_dir/core/build/dependant-libs-${SCALA_VERSION}*;
do
CLASSPATH=$CLASSPATH:$dir/*
if [ -z $CLASSPATH ] ; then
CLASSPATH=$dir/*
else
CLASSPATH=$CLASSPATH:$dir/*
fi
done

for file in $base_dir/examples/build/libs//kafka-examples*.jar;
Expand Down
1 change: 1 addition & 0 deletions checkstyle/import-control.xml
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,7 @@

<subpackage name="util">
<allow pkg="org.apache.kafka.connect" />
<allow pkg="org.reflections.vfs" />
<!-- for annotations to avoid code duplication -->
<allow pkg="com.fasterxml.jackson.annotation" />
</subpackage>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.kafka.connect.tools.VerifiableSinkConnector;
import org.apache.kafka.connect.tools.VerifiableSourceConnector;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.apache.kafka.connect.util.ReflectionsUtil;
import org.reflections.Reflections;
import org.reflections.util.ClasspathHelper;
import org.reflections.util.ConfigurationBuilder;
Expand Down Expand Up @@ -82,11 +83,10 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
protected final ConfigBackingStore configBackingStore;

private Map<String, Connector> tempConnectors = new ConcurrentHashMap<>();
private static final List<Class<? extends Connector>> SKIPPED_CONNECTORS = Arrays.<Class<? extends Connector>>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class);
private static List<ConnectorPluginInfo> validConnectorPlugins;
private static final Object LOCK = new Object();
private Thread classPathTraverser;

private static final List<Class<? extends Connector>> EXCLUDES = Arrays.<Class<? extends Connector>>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class);

public AbstractHerder(Worker worker,
String workerId,
Expand Down Expand Up @@ -263,10 +263,12 @@ public static List<ConnectorPluginInfo> connectorPlugins() {
if (validConnectorPlugins != null) {
return validConnectorPlugins;
}
ReflectionsUtil.registerUrlTypes();
ConfigurationBuilder builder = new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath());
Reflections reflections = new Reflections(builder);

Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath()));
Set<Class<? extends Connector>> connectorClasses = reflections.getSubTypesOf(Connector.class);
connectorClasses.removeAll(SKIPPED_CONNECTORS);
connectorClasses.removeAll(EXCLUDES);
List<ConnectorPluginInfo> connectorPlugins = new LinkedList<>();
for (Class<? extends Connector> connectorClass : connectorClasses) {
int mod = connectorClass.getModifiers();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.util;

import org.reflections.vfs.Vfs;
import org.reflections.vfs.Vfs.Dir;
import org.reflections.vfs.Vfs.File;
import org.reflections.vfs.Vfs.UrlType;

import java.net.URL;
import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;

/**
* CLASSPATH on OSX contains .mar and .jnilib file extensions. Vfs used by Reflections does not recognize
* urls with those extensions and log WARNs when scan them. Those WARNs can be eliminated by registering
* URL types before using reflection.
*/
public class ReflectionsUtil {

private static final String FILE_PROTOCOL = "file";
private static final List<String> ENDINGS = Arrays.asList(".mar", ".jnilib", "*");

public static void registerUrlTypes() {
final List<UrlType> urlTypes = new LinkedList<>();
urlTypes.add(new EmptyUrlType(ENDINGS));
urlTypes.addAll(Arrays.asList(Vfs.DefaultUrlTypes.values()));
Vfs.setDefaultURLTypes(urlTypes);
}

private static class EmptyUrlType implements UrlType {

private final List<String> endings;

private EmptyUrlType(final List<String> endings) {
this.endings = endings;
}

public boolean matches(URL url) {
final String protocol = url.getProtocol();
final String externalForm = url.toExternalForm();
if (!protocol.equals(FILE_PROTOCOL)) {
return false;
}
for (String ending : endings) {
if (externalForm.endsWith(ending)) {
return true;
}
}
return false;
}

public Dir createDir(final URL url) throws Exception {
return emptyVfsDir(url);
}

private static Dir emptyVfsDir(final URL url) {
return new Dir() {
@Override
public String getPath() {
return url.toExternalForm();
}

@Override
public Iterable<File> getFiles() {
return Collections.emptyList();
}

@Override
public void close() {

}
};
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@

import org.apache.kafka.common.config.Config;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Recommender;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Recommender;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigDef.Width;
import org.apache.kafka.connect.connector.Connector;
Expand Down Expand Up @@ -149,6 +149,7 @@ public void testListConnectorPlugins() {
assertTrue(connectorPlugins.contains(new ConnectorPluginInfo(ConnectorPluginsResourceTestConnector.class.getCanonicalName())));
}


/* Name here needs to be unique as we are testing the aliasing mechanism */
public static class ConnectorPluginsResourceTestConnector extends Connector {

Expand Down