Skip to content
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

[BEAM-14048] [CdapIO] Add ConfigWrapper for building CDAP PluginConfigs #17051

Merged
merged 23 commits into from
May 5, 2022
Merged
Show file tree
Hide file tree
Changes from 14 commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
78dd87b
[BEAM-14048] Add ConfigWrapper for building CDAP PluginConfigs
Amar3tto Mar 9, 2022
30b9e38
[BEAM-14048] Fix checkstyle
Amar3tto Mar 11, 2022
9e5e6d6
[BEAM-14048] Fix warnings
Amar3tto Mar 11, 2022
2d414f4
[BEAM-14048] Fix warnings
Amar3tto Mar 15, 2022
d35acb8
[BEAM-14048] Fix warning
Amar3tto Mar 15, 2022
b7e0fca
[BEAM-14048] Fix warning
Amar3tto Mar 15, 2022
3252395
[BEAM-14048] Remove unused dependencies
Amar3tto Mar 15, 2022
f9e5675
[BEAM-14048] Add needed dependencies
Amar3tto Mar 15, 2022
65a5442
[BEAM-14048] Fix spotless
Amar3tto Mar 15, 2022
00aef64
[BEAM-14048] Fix typo
Amar3tto Mar 15, 2022
d10b84b
[BEAM-14048] Use fori instead of stream
Amar3tto Mar 16, 2022
2f9686d
[BEAM-14048] Suppress warning
Amar3tto Mar 16, 2022
119d2cc
[BEAM-14048] Add used undeclared artifacts
Amar3tto Mar 16, 2022
1f63d65
[BEAM-14048] Change dependencies to test
Amar3tto Mar 16, 2022
0645c7b
[BEAM-14048] Refactoring
Amar3tto Mar 22, 2022
44125f0
Merge branch 'apache:master' into BEAM-14048-cdap-config
AKosolapov Mar 23, 2022
8ba78c9
[BEAM-14048] Use CDAP InstantiatorFactory for creating config objects
Amar3tto Mar 30, 2022
970fe79
[BEAM-14048] Suppress warning
Amar3tto Mar 30, 2022
d7d2042
Update maven repo
elizaveta-lomteva Apr 18, 2022
129cffe
Update build.gradle
elizaveta-lomteva Apr 18, 2022
f440da9
[BEAM-14048] Use ServiceNow CDAP dependency from Maven central
Amar3tto May 4, 2022
7c0028c
Merge remote-tracking branch 'origin/BEAM-14048-cdap-config' into BEA…
Amar3tto May 4, 2022
9df1c75
[BEAM-14048] Set macroFields
Amar3tto May 4, 2022
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
1 change: 1 addition & 0 deletions sdks/java/io/cdap/OWNERS
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
# See the OWNERS docs at https://s.apache.org/beam-owners
69 changes: 69 additions & 0 deletions sdks/java/io/cdap/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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.
*/

plugins {
id 'java'
id 'org.apache.beam.module'
}

applyJavaNature(
exportJavadoc: false,
automaticModuleName: 'org.apache.beam.sdk.io.cdap',
)
provideIntegrationTestingDependencies()
enableJavaPerformanceTesting()

description = "Apache Beam :: CDAP :: Java"
ext.summary = """Apache Beam SDK provides a simple, Java-based
interface for integration with CDAP plugins."""

/** Define the list of runners which execute a precommit test.
* Some runners are run from separate projects, see the preCommit task below
* for details.
*/

allprojects {
repositories {
maven { url 'https://jitpack.io' }
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why was this needed ?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a temporary solution until we deal with the vendored dependencies. (e.x. com.github.data-integrations:salesforce)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we wait till dependency issue is resolved before committing the code ?
How would this work for release Beam ? Sounds like this just builds a local SNAPSHOT jar and probably not good as a permanent dependency for Beam.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree that it's better to wait for the plugins to be published in the Maven Central Repository.

}
}

dependencies {
implementation project(path: ":sdks:java:core", configuration: "shadow")
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
implementation library.java.slf4j_api
implementation "io.cdap.cdap:cdap-api:6.5.1"
implementation "com.fasterxml.jackson.core:jackson-core:2.13.0"
implementation "com.fasterxml.jackson.core:jackson-databind:2.13.0"
testImplementation library.java.vendored_guava_26_0_jre
testImplementation "com.github.data-integrations:salesforce:v1.3.9"
testImplementation "io.cdap.cdap:cdap-etl-api:6.5.1"
testImplementation project(path: ":sdks:java:core", configuration: "shadowTest")
testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration")
testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration")
testImplementation project(":sdks:java:io:jdbc")
testImplementation project(path: ":examples:java", configuration: "testRuntimeMigration")
testImplementation project(path: ":runners:direct-java", configuration: "shadow")
testImplementation project(":sdks:java:io:google-cloud-platform")
testImplementation project(":sdks:java:extensions:ml")
testImplementation library.java.google_cloud_bigquery
testImplementation library.java.hamcrest
testImplementation library.java.junit
testImplementation library.java.mockito_core
testImplementation library.java.testcontainers_kafka
testImplementation library.java.testcontainers_gcloud
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* 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.beam.sdk.io.cdap;

import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.cdap.cdap.api.plugin.PluginConfig;
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/** Class for building {@link PluginConfig} object of the specific class {@param <T>}. */
public class ConfigWrapper<T extends PluginConfig> {

private static final Logger LOG = LoggerFactory.getLogger(ConfigWrapper.class);

@Nullable private Map<String, Object> paramsMap = null;
private final Class<T> configClass;

public ConfigWrapper(Class<T> configClass) {
this.configClass = configClass;
}

public ConfigWrapper<T> fromJsonString(String jsonString) {
TypeReference<HashMap<String, Object>> typeRef =
new TypeReference<HashMap<String, Object>>() {};
try {
paramsMap = new ObjectMapper().readValue(jsonString, typeRef);
} catch (IOException e) {
LOG.error("Can not read json string to params map", e);
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
}
return this;
}

public ConfigWrapper<T> fromJsonFile(File jsonFile) {
TypeReference<HashMap<String, Object>> typeRef =
new TypeReference<HashMap<String, Object>>() {};
try {
paramsMap = new ObjectMapper().readValue(jsonFile, typeRef);
} catch (IOException e) {
LOG.error("Can not read json file to params map", e);
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
}
return this;
}

public ConfigWrapper<T> withParams(Map<String, Object> paramsMap) {
this.paramsMap = new HashMap<>(paramsMap);
return this;
}

public ConfigWrapper<T> setParam(String paramName, Object param) {
getParamsMap().put(paramName, param);
return this;
}

public @Nullable T build() {
return PluginConfigInstantiationUtils.getPluginConfig(getParamsMap(), configClass);
}

private @Nonnull Map<String, Object> getParamsMap() {
if (paramsMap == null) {
paramsMap = new HashMap<>();
}
return paramsMap;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
/*
* 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.beam.sdk.io.cdap;

import io.cdap.cdap.api.annotation.Name;
import io.cdap.cdap.api.plugin.PluginConfig;
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Modifier;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Class for getting any filled {@link io.cdap.cdap.api.plugin.PluginConfig} configuration object.
*/
@SuppressWarnings({"unchecked", "assignment.type.incompatible"})
public class PluginConfigInstantiationUtils {

private static final Logger LOG = LoggerFactory.getLogger(PluginConfigInstantiationUtils.class);

/**
* @param params map of config fields, where key is the name of the field, value must be String or
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
* boxed primitive
* @return Config object for given map of arguments and configuration class
*/
public static @Nullable <T extends PluginConfig> T getPluginConfig(
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
Map<String, Object> params, Class<T> configClass) {
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
// Validate configClass
if (configClass == null || configClass.isPrimitive() || configClass.isArray()) {
throw new IllegalArgumentException("Config class must be correct!");
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
}
List<Field> allFields = new ArrayList<>();
Class<?> currClass = configClass;
while (currClass != null && !currClass.equals(Object.class)) {
allFields.addAll(
Arrays.stream(currClass.getDeclaredFields())
.filter(
f -> !Modifier.isStatic(f.getModifiers()) && f.isAnnotationPresent(Name.class))
.collect(Collectors.toList()));
currClass = currClass.getSuperclass();
}
T config = getEmptyObjectOf(configClass);

if (config != null) {
for (Field field : allFields) {
field.setAccessible(true);

Class<?> fieldType = field.getType();

Name declaredAnnotation = field.getDeclaredAnnotation(Name.class);
Object fieldValue =
declaredAnnotation != null ? params.get(declaredAnnotation.value()) : null;

if (fieldValue != null && fieldType.equals(fieldValue.getClass())) {
try {
field.set(config, fieldValue);
} catch (IllegalAccessException e) {
LOG.error("Can not set a field", e);
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
}
}
}
}
return config;
}

/** @return empty {@link Object} of {@param tClass} */
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
private static @Nullable <T> T getEmptyObjectOf(Class<T> tClass) {
for (Constructor<?> constructor : tClass.getDeclaredConstructors()) {
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
constructor.setAccessible(true);
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
Class<?>[] parameterTypes = constructor.getParameterTypes();
Object[] parameters = new Object[parameterTypes.length];
for (int i = 0; i < parameterTypes.length; i++) {
parameters[i] = getDefaultValue(parameterTypes[i]);
}
try {
return (T) constructor.newInstance(parameters);
} catch (InstantiationException | IllegalAccessException | InvocationTargetException e) {
LOG.error("Can not instantiate an empty object", e);
}
}
return null;
}

/** @return default value for given {@param tClass} */
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
private static @Nullable Object getDefaultValue(@Nullable Class<?> tClass) {
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
if (Boolean.TYPE.equals(tClass)) {
return false;
}
if (Character.TYPE.equals(tClass)) {
return Character.MIN_VALUE;
}
if (Byte.TYPE.equals(tClass)) {
return Byte.MIN_VALUE;
}
if (Short.TYPE.equals(tClass)) {
return Short.MIN_VALUE;
}
if (Double.TYPE.equals(tClass)) {
return Double.MIN_VALUE;
}
if (Integer.TYPE.equals(tClass)) {
return Integer.MIN_VALUE;
}
if (Float.TYPE.equals(tClass)) {
return Float.MIN_VALUE;
}
if (Long.TYPE.equals(tClass)) {
return Long.MIN_VALUE;
}
return null;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
/*
* 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.
*/

/** Transforms for reading and writing from CDAP. */
@Experimental(Kind.SOURCE_SINK)
package org.apache.beam.sdk.io.cdap;

import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.annotations.Experimental.Kind;
Loading