Skip to content

Commit

Permalink
Add a new api to get information about config definition of builtin s…
Browse files Browse the repository at this point in the history
…ources/sinks (apache#7114)

Co-authored-by: Sanjeev Kulkarni <[email protected]>
  • Loading branch information
srkukarni and Sanjeev Kulkarni authored Jun 2, 2020
1 parent 05d18c5 commit f0b6934
Show file tree
Hide file tree
Showing 13 changed files with 296 additions and 64 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.commons.lang.StringUtils;
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.common.functions.UpdateOptions;
import org.apache.pulsar.common.io.ConfigFieldDefinition;
import org.apache.pulsar.common.io.ConnectorDefinition;
import org.apache.pulsar.common.io.SinkConfig;
import org.apache.pulsar.common.policies.data.SinkStatus;
Expand Down Expand Up @@ -478,14 +479,27 @@ public void startSink(@ApiParam(value = "The tenant of a Pulsar Sink")
})
@Path("/builtinsinks")
public List<ConnectorDefinition> getSinkList() {
List<ConnectorDefinition> connectorDefinitions = sink.getListOfConnectors();
List<ConnectorDefinition> retval = new ArrayList<>();
for (ConnectorDefinition connectorDefinition : connectorDefinitions) {
if (!StringUtils.isEmpty(connectorDefinition.getSinkClass())) {
retval.add(connectorDefinition);
}
}
return retval;
return sink.getSinkList();
}

@GET
@ApiOperation(
value = "Fetches information about config fields associated with the specified builtin sink",
response = ConfigFieldDefinition.class,
responseContainer = "List"
)
@ApiResponses(value = {
@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"),
@ApiResponse(code = 404, message = "builtin sink does not exist"),
@ApiResponse(code = 500, message = "Internal server error"),
@ApiResponse(code = 503, message = "Function worker service is now initializing. Please try again later.")
})
@Produces(MediaType.APPLICATION_JSON)
@Path("/builtinsinks/{name}/configdefinition")
public List<ConfigFieldDefinition> getSinkConfigDefinition(
@ApiParam(value = "The name of the builtin sink")
final @PathParam("name") String name) throws IOException {
return sink.getSinkConfigDefinition(name);
}

@POST
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.commons.lang.StringUtils;
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.common.functions.UpdateOptions;
import org.apache.pulsar.common.io.ConfigFieldDefinition;
import org.apache.pulsar.common.io.ConnectorDefinition;
import org.apache.pulsar.common.io.SourceConfig;
import org.apache.pulsar.common.policies.data.SourceStatus;
Expand Down Expand Up @@ -468,14 +469,27 @@ public void startSource(
@Produces(MediaType.APPLICATION_JSON)
@Path("/builtinsources")
public List<ConnectorDefinition> getSourceList() {
List<ConnectorDefinition> connectorDefinitions = source.getListOfConnectors();
List<ConnectorDefinition> retval = new ArrayList<>();
for (ConnectorDefinition connectorDefinition : connectorDefinitions) {
if (!StringUtils.isEmpty(connectorDefinition.getSourceClass())) {
retval.add(connectorDefinition);
}
}
return retval;
return source.getSourceList();
}

@GET
@ApiOperation(
value = "Fetches information about config fields associated with the specified builtin source",
response = ConfigFieldDefinition.class,
responseContainer = "List"
)
@ApiResponses(value = {
@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"),
@ApiResponse(code = 404, message = "builtin source does not exist"),
@ApiResponse(code = 500, message = "Internal server error"),
@ApiResponse(code = 503, message = "Function worker service is now initializing. Please try again later.")
})
@Produces(MediaType.APPLICATION_JSON)
@Path("/builtinsources/{name}/configdefinition")
public List<ConfigFieldDefinition> getSourceConfigDefinition(
@ApiParam(value = "The name of the builtin source")
final @PathParam("name") String name) throws IOException {
return source.getSourceConfigDefinition(name);
}

@POST
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/**
* 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.pulsar.common.io;

import java.util.Map;

import lombok.Data;
import lombok.NoArgsConstructor;

/**
* Information about a Pulsar connector config field.
*/
@Data
@NoArgsConstructor
public class ConfigFieldDefinition {

/**
* The name of the field.
*/
private String fieldName;

/**
* The field type.
*/
private String typeName;

/**
* Other attribute pairs associated with the field.
*/
private Map<String, String> attributes;
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,13 @@
import java.io.IOException;
import java.lang.reflect.Array;
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.net.URLClassLoader;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;

Expand Down Expand Up @@ -304,4 +308,15 @@ public static Class loadClass(String className, ClassLoader classLoader) throws
}
}
}

public static List<Field> getAllFields(Class<?> type) {
List<Field> fields = new LinkedList<>();
fields.addAll(Arrays.asList(type.getDeclaredFields()));

if (type.getSuperclass() != null) {
fields.addAll(getAllFields(type.getSuperclass()));
}

return fields;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,22 +20,28 @@

import java.io.File;
import java.io.IOException;
import java.lang.annotation.Annotation;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Collections;
import java.util.*;

import lombok.experimental.UtilityClass;
import lombok.extern.slf4j.Slf4j;

import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.common.io.ConfigFieldDefinition;
import org.apache.pulsar.common.io.ConnectorDefinition;
import org.apache.pulsar.common.nar.NarClassLoader;
import org.apache.pulsar.common.util.ObjectMapperFactory;
import org.apache.pulsar.common.util.Reflections;
import org.apache.pulsar.functions.utils.Exceptions;
import org.apache.pulsar.io.core.Sink;
import org.apache.pulsar.io.core.Source;
import org.apache.pulsar.io.core.annotations.FieldDoc;

@UtilityClass
@Slf4j
Expand Down Expand Up @@ -108,6 +114,38 @@ public static ConnectorDefinition getConnectorDefinition(ClassLoader classLoader
return ObjectMapperFactory.getThreadLocalYaml().readValue(configStr, ConnectorDefinition.class);
}

public static List<ConfigFieldDefinition> getConnectorConfigDefinition(String narPath,
String configClassName,
String narExtractionDirectory) throws Exception {
List<ConfigFieldDefinition> retval = new LinkedList<>();
try (NarClassLoader ncl = NarClassLoader.getFromArchive(new File(narPath), Collections.emptySet(), narExtractionDirectory)) {
Class configClass = ncl.loadClass(configClassName);
for (Field field : Reflections.getAllFields(configClass)) {
if (java.lang.reflect.Modifier.isStatic(field.getModifiers())) {
// We dont want static fields
continue;
}
field.setAccessible(true);
ConfigFieldDefinition configFieldDefinition = new ConfigFieldDefinition();
configFieldDefinition.setFieldName(field.getName());
configFieldDefinition.setTypeName(field.getType().getName());
Map<String, String> attributes = new HashMap<>();
for (Annotation annotation : field.getAnnotations()) {
if (annotation.annotationType().equals(FieldDoc.class)) {
FieldDoc fieldDoc = (FieldDoc) annotation;
for (Method method : FieldDoc.class.getDeclaredMethods()) {
Object value = method.invoke(fieldDoc);
attributes.put(method.getName(), value == null ? "" : value.toString());
}
}
}
configFieldDefinition.setAttributes(attributes);
retval.add(configFieldDefinition);
}
}
return retval;
}

public static Connectors searchForConnectors(String connectorsDirectory, String narExtractionDirectory) throws IOException {
Path path = Paths.get(connectorsDirectory).toAbsolutePath();
log.info("Searching for connectors in {}", path);
Expand All @@ -127,10 +165,16 @@ public static Connectors searchForConnectors(String connectorsDirectory, String

if (!StringUtils.isEmpty(cntDef.getSourceClass())) {
connectors.sources.put(cntDef.getName(), archive);
if (!StringUtils.isEmpty(cntDef.getSourceConfigClass())) {
connectors.sourceConfigDefinitions.put(cntDef.getName(), getConnectorConfigDefinition(archive.toString(), cntDef.getSourceConfigClass(), narExtractionDirectory));
}
}

if (!StringUtils.isEmpty(cntDef.getSinkClass())) {
connectors.sinks.put(cntDef.getName(), archive);
if (!StringUtils.isEmpty(cntDef.getSinkConfigClass())) {
connectors.sinkConfigDefinitions.put(cntDef.getName(), getConnectorConfigDefinition(archive.toString(), cntDef.getSinkConfigClass(), narExtractionDirectory));
}
}

connectors.connectors.add(cntDef);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,14 @@

import lombok.Data;

import org.apache.pulsar.common.io.ConfigFieldDefinition;
import org.apache.pulsar.common.io.ConnectorDefinition;

@Data
public class Connectors {
final List<ConnectorDefinition> connectors = new ArrayList<>();
final Map<String, Path> sources = new TreeMap<>();
final Map<String, List<ConfigFieldDefinition>> sourceConfigDefinitions = new TreeMap<>();
final Map<String, Path> sinks = new TreeMap<>();
final Map<String, List<ConfigFieldDefinition>> sinkConfigDefinitions = new TreeMap<>();
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.nio.file.Path;
import java.util.List;

import org.apache.pulsar.common.io.ConfigFieldDefinition;
import org.apache.pulsar.common.io.ConnectorDefinition;
import org.apache.pulsar.functions.utils.io.ConnectorUtils;
import org.apache.pulsar.functions.utils.io.Connectors;
Expand All @@ -42,6 +43,14 @@ public Path getSourceArchive(String sourceType) {
return connectors.getSources().get(sourceType);
}

public List<ConfigFieldDefinition> getSourceConfigDefinition(String sourceType) {
return connectors.getSourceConfigDefinitions().get(sourceType);
}

public List<ConfigFieldDefinition> getSinkConfigDefinition(String sinkType) {
return connectors.getSinkConfigDefinitions().get(sinkType);
}

public Path getSinkArchive(String sinkType) {
return connectors.getSinks().get(sinkType);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.common.functions.UpdateOptions;
import org.apache.pulsar.common.functions.Utils;
import org.apache.pulsar.common.io.ConfigFieldDefinition;
import org.apache.pulsar.common.io.ConnectorDefinition;
import org.apache.pulsar.common.io.SinkConfig;
import org.apache.pulsar.common.policies.data.ExceptionInformation;
import org.apache.pulsar.common.policies.data.SinkStatus;
Expand All @@ -49,10 +51,7 @@
import java.io.InputStream;
import java.net.URI;
import java.nio.file.Path;
import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
import java.util.Optional;
import java.util.*;
import java.util.function.Supplier;

import static org.apache.commons.lang3.StringUtils.isBlank;
Expand Down Expand Up @@ -665,6 +664,28 @@ public SinkConfig getSinkInfo(final String tenant,
return config;
}

public List<ConnectorDefinition> getSinkList() {
List<ConnectorDefinition> connectorDefinitions = getListOfConnectors();
List<ConnectorDefinition> retval = new ArrayList<>();
for (ConnectorDefinition connectorDefinition : connectorDefinitions) {
if (!org.apache.commons.lang.StringUtils.isEmpty(connectorDefinition.getSinkClass())) {
retval.add(connectorDefinition);
}
}
return retval;
}

public List<ConfigFieldDefinition> getSinkConfigDefinition(String name) {
if (!isWorkerServiceAvailable()) {
throwUnavailableException();
}
List<ConfigFieldDefinition> retval = this.worker().getConnectorsManager().getSinkConfigDefinition(name);
if (retval == null) {
throw new RestException(Response.Status.NOT_FOUND, "builtin sink does not exist");
}
return retval;
}

private Function.FunctionDetails validateUpdateRequestParams(final String tenant,
final String namespace,
final String sinkName,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.common.functions.UpdateOptions;
import org.apache.pulsar.common.functions.Utils;
import org.apache.pulsar.common.io.ConfigFieldDefinition;
import org.apache.pulsar.common.io.ConnectorDefinition;
import org.apache.pulsar.common.io.SourceConfig;
import org.apache.pulsar.common.policies.data.ExceptionInformation;
import org.apache.pulsar.common.policies.data.SourceStatus;
Expand All @@ -49,10 +51,7 @@
import java.io.InputStream;
import java.net.URI;
import java.nio.file.Path;
import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
import java.util.Optional;
import java.util.*;
import java.util.function.Supplier;

import static org.apache.commons.lang3.StringUtils.isBlank;
Expand Down Expand Up @@ -661,6 +660,28 @@ public SourceConfig getSourceInfo(final String tenant,
return config;
}

public List<ConnectorDefinition> getSourceList() {
List<ConnectorDefinition> connectorDefinitions = getListOfConnectors();
List<ConnectorDefinition> retval = new ArrayList<>();
for (ConnectorDefinition connectorDefinition : connectorDefinitions) {
if (!org.apache.commons.lang.StringUtils.isEmpty(connectorDefinition.getSourceClass())) {
retval.add(connectorDefinition);
}
}
return retval;
}

public List<ConfigFieldDefinition> getSourceConfigDefinition(String name) {
if (!isWorkerServiceAvailable()) {
throwUnavailableException();
}
List<ConfigFieldDefinition> retval = this.worker().getConnectorsManager().getSourceConfigDefinition(name);
if (retval == null) {
throw new RestException(Response.Status.NOT_FOUND, "builtin source does not exist");
}
return retval;
}

private Function.FunctionDetails validateUpdateRequestParams(final String tenant,
final String namespace,
final String sourceName,
Expand Down
Loading

0 comments on commit f0b6934

Please sign in to comment.