From 78dd87b08b1519c15ffb5f2e11821fd0ab22ad71 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 9 Mar 2022 18:21:20 +0400 Subject: [PATCH 01/31] [BEAM-14048] Add ConfigWrapper for building CDAP PluginConfigs --- sdks/java/io/cdap/OWNERS | 1 + sdks/java/io/cdap/build.gradle | 79 +++++++++++ .../ConfigWrapper.java | 84 +++++++++++ .../PluginConfigInstantiationUtils.java | 129 +++++++++++++++++ .../ConfigWrapperTest.java | 131 ++++++++++++++++++ .../resources/salesforce_test_params.json | 10 ++ settings.gradle.kts | 1 + 7 files changed, 435 insertions(+) create mode 100644 sdks/java/io/cdap/OWNERS create mode 100644 sdks/java/io/cdap/build.gradle create mode 100644 sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/ConfigWrapper.java create mode 100644 sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/PluginConfigInstantiationUtils.java create mode 100644 sdks/java/io/cdap/src/test/java/org.apache.beam.sdk.io.cdap/ConfigWrapperTest.java create mode 100644 sdks/java/io/cdap/src/test/resources/salesforce_test_params.json diff --git a/sdks/java/io/cdap/OWNERS b/sdks/java/io/cdap/OWNERS new file mode 100644 index 000000000000..e60eec69932b --- /dev/null +++ b/sdks/java/io/cdap/OWNERS @@ -0,0 +1 @@ +# See the OWNERS docs at https://s.apache.org/beam-owners diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle new file mode 100644 index 000000000000..80143c428488 --- /dev/null +++ b/sdks/java/io/cdap/build.gradle @@ -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. + */ + +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' } + } +} + +dependencies { + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:io:hadoop-common") + implementation project(":sdks:java:io:hadoop-format") + implementation "org.apache.hadoop:hadoop-mapred:0.22.0" + implementation library.java.vendored_guava_26_0_jre + implementation library.java.slf4j_api + implementation "org.apache.commons:commons-lang3:3.9" + implementation "org.apache.httpcomponents:httpclient:4.5.13" + implementation "org.apache.httpcomponents:httpcore:4.4.13" + implementation "io.cdap.cdap:cdap-api:6.5.1" + implementation "io.cdap.cdap:cdap-formats:6.5.1" + implementation "io.cdap.cdap:cdap-etl-api:6.5.1" + implementation "com.github.data-integrations:salesforce:v1.3.9" + implementation 'com.google.http-client:google-http-client-gson:1.32.1' + implementation 'com.github.rholder:guava-retrying:2.0.0' + implementation 'commons-validator:commons-validator:1.7' + 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(":sdks:java:io:hadoop-common") + testImplementation project(":sdks:java:io:hadoop-format") + 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 +} diff --git a/sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/ConfigWrapper.java b/sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/ConfigWrapper.java new file mode 100644 index 000000000000..b32f6c2241e4 --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/ConfigWrapper.java @@ -0,0 +1,84 @@ +/* + * 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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Class for building {@link PluginConfig} object of the specific class {@param }. */ +public class ConfigWrapper { + + private static final Logger LOG = LoggerFactory.getLogger(ConfigWrapper.class); + + private Map paramsMap; + private final Class configClass; + + public ConfigWrapper(Class configClass) { + this.configClass = configClass; + } + + public ConfigWrapper fromJsonString(String jsonString) { + TypeReference> typeRef = + new TypeReference>() {}; + try { + paramsMap = new ObjectMapper().readValue(jsonString, typeRef); + } catch (IOException e) { + LOG.error("Can not read json string to params map", e); + } + return this; + } + + public ConfigWrapper fromJsonFile(File jsonFile) { + TypeReference> typeRef = + new TypeReference>() {}; + try { + paramsMap = new ObjectMapper().readValue(jsonFile, typeRef); + } catch (IOException e) { + LOG.error("Can not read json file to params map", e); + } + return this; + } + + public ConfigWrapper withParams(Map paramsMap) { + this.paramsMap = new HashMap<>(paramsMap); + return this; + } + + public ConfigWrapper setParam(String paramName, Object param) { + getParamsMap().put(paramName, param); + return this; + } + + public T build() { + return PluginConfigInstantiationUtils.getPluginConfig(getParamsMap(), configClass); + } + + private Map getParamsMap() { + if (paramsMap == null) { + paramsMap = new HashMap<>(); + } + return paramsMap; + } +} diff --git a/sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/PluginConfigInstantiationUtils.java new file mode 100644 index 000000000000..f60dab39e193 --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/PluginConfigInstantiationUtils.java @@ -0,0 +1,129 @@ +/* + * 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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Class for getting any filled {@link io.cdap.cdap.api.plugin.PluginConfig} configuration object. + */ +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 + * boxed primitive + * @return Config object for given map of arguments and configuration class + */ + public static T getPluginConfig( + Map params, Class configClass) { + // Validate configClass + if (configClass == null || configClass.isPrimitive() || configClass.isArray()) { + throw new IllegalArgumentException("Config class must be correct!"); + } + List allFields = new ArrayList<>(); + Class currClass = configClass; + while (!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); + + for (Field field : allFields) { + field.setAccessible(true); + + Class fieldType = field.getType(); + + String fieldName = field.getDeclaredAnnotation(Name.class).value(); + Object fieldValue = params.get(fieldName); + + if (fieldValue != null && fieldType.equals(fieldValue.getClass())) { + try { + field.set(config, fieldValue); + } catch (IllegalAccessException e) { + LOG.error("Can not set a field", e); + } + } + } + return config; + } + + /** @return empty {@link Object} of {@param tClass} */ + private static T getEmptyObjectOf(Class tClass) { + for (Constructor constructor : tClass.getDeclaredConstructors()) { + constructor.setAccessible(true); + Class[] parameterTypes = constructor.getParameterTypes(); + Object[] parameters = + Arrays.stream(parameterTypes) + .map(PluginConfigInstantiationUtils::getDefaultValue) + .toArray(); + 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} */ + private static Object getDefaultValue(Class tClass) { + 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; + } +} diff --git a/sdks/java/io/cdap/src/test/java/org.apache.beam.sdk.io.cdap/ConfigWrapperTest.java b/sdks/java/io/cdap/src/test/java/org.apache.beam.sdk.io.cdap/ConfigWrapperTest.java new file mode 100644 index 000000000000..82afa041ad7f --- /dev/null +++ b/sdks/java/io/cdap/src/test/java/org.apache.beam.sdk.io.cdap/ConfigWrapperTest.java @@ -0,0 +1,131 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import io.cdap.plugin.salesforce.SalesforceConstants; +import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceSourceConfig; +import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; +import java.io.File; +import java.util.Map; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; + +/** Test class for {@link ConfigWrapper}. */ +@RunWith(JUnit4.class) +public class ConfigWrapperTest { + + private static final Logger LOG = LoggerFactory.getLogger(ConfigWrapperTest.class); + + private static final ImmutableMap TEST_SALESFORCE_PARAMS_MAP = + ImmutableMap.builder() + .put("sObjectName", "sObject") + .put("datetimeAfter", "datetime") + .put("consumerKey", "key") + .put("consumerSecret", "secret") + .put("username", "user") + .put("password", "password") + .put("loginUrl", "https://www.google.com") + .put("referenceName", "oldReference") + .build(); + private static final String TEST_SALESFORCE_PARAMS_JSON_STRING = + "{\n" + + "\"sObjectName\": \"sObject\",\n" + + "\"datetimeAfter\": \"datetime\",\n" + + "\"consumerKey\": \"key\",\n" + + "\"consumerSecret\": \"secret\",\n" + + "\"username\": \"user\",\n" + + "\"password\": \"password\",\n" + + "\"loginUrl\": \"https://www.google.com\",\n" + + "\"referenceName\": \"reference\"\n" + + "}"; + private static final String SALESFORCE_TEST_PARAMS_JSON = + "src/test/resources/salesforce_test_params.json"; + public static final String REFERENCE_NAME_PARAM_NAME = "referenceName"; + + @Test + public void testBuildingPluginConfigFromParamsMap() { + try { + String newReferenceName = "new reference name"; + SalesforceSourceConfig firstConfig = + new ConfigWrapper<>(SalesforceSourceConfig.class) + .withParams(TEST_SALESFORCE_PARAMS_MAP) + .setParam("referenceName", newReferenceName) + .build(); + validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, firstConfig); + assertEquals(newReferenceName, firstConfig.referenceName); + } catch (Exception e) { + LOG.error("Error occurred while building the config object", e); + fail(); + } + } + + @Test + public void testBuildingPluginConfigFromJsonFile() { + try { + String newReferenceName = "new reference name"; + SalesforceSourceConfig secondConfig = + new ConfigWrapper<>(SalesforceSourceConfig.class) + .fromJsonFile(new File(SALESFORCE_TEST_PARAMS_JSON)) + .setParam(REFERENCE_NAME_PARAM_NAME, newReferenceName) + .build(); + validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, secondConfig); + assertEquals(newReferenceName, secondConfig.referenceName); + } catch (Exception e) { + LOG.error("Error occurred while building the config object", e); + fail(); + } + } + + @Test + public void testBuildingPluginConfigFromJsonString() { + try { + String newReferenceName = "new reference name"; + SalesforceSourceConfig secondConfig = + new ConfigWrapper<>(SalesforceSourceConfig.class) + .fromJsonString(TEST_SALESFORCE_PARAMS_JSON_STRING) + .setParam(REFERENCE_NAME_PARAM_NAME, newReferenceName) + .build(); + validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, secondConfig); + assertEquals(newReferenceName, secondConfig.referenceName); + } catch (Exception e) { + LOG.error("Error occurred while building the config object", e); + fail(); + } + } + + private static void validateSalesforceConfigObject( + Map params, SalesforceSourceConfig config) { + assertEquals( + params.get(SalesforceSourceConstants.PROPERTY_DATETIME_AFTER), config.getDatetimeAfter()); + assertEquals( + params.get(SalesforceSourceConstants.PROPERTY_SOBJECT_NAME), config.getSObjectName()); + assertEquals(params.get(SalesforceConstants.PROPERTY_CONSUMER_KEY), config.getConsumerKey()); + assertEquals( + params.get(SalesforceConstants.PROPERTY_CONSUMER_SECRET), config.getConsumerSecret()); + assertEquals(params.get(SalesforceConstants.PROPERTY_USERNAME), config.getUsername()); + assertEquals(params.get(SalesforceConstants.PROPERTY_PASSWORD), config.getPassword()); + assertEquals(params.get(SalesforceConstants.PROPERTY_LOGIN_URL), config.getLoginUrl()); + } +} diff --git a/sdks/java/io/cdap/src/test/resources/salesforce_test_params.json b/sdks/java/io/cdap/src/test/resources/salesforce_test_params.json new file mode 100644 index 000000000000..bd9891e35078 --- /dev/null +++ b/sdks/java/io/cdap/src/test/resources/salesforce_test_params.json @@ -0,0 +1,10 @@ +{ + "sObjectName": "sObject", + "datetimeAfter": "datetime", + "consumerKey": "key", + "consumerSecret": "secret", + "username": "user", + "password": "password", + "loginUrl": "https://www.google.com", + "referenceName": "reference" +} diff --git a/settings.gradle.kts b/settings.gradle.kts index 6796ad0b09d7..9b0f4109006c 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -165,6 +165,7 @@ include(":sdks:java:io:elasticsearch-tests:elasticsearch-tests-common") include(":sdks:java:io:expansion-service") include(":sdks:java:io:file-based-io-tests") include(":sdks:java:io:bigquery-io-perf-tests") +include(":sdks:java:io:cdap") include(":sdks:java:io:google-cloud-platform") include(":sdks:java:io:google-cloud-platform:expansion-service") include(":sdks:java:io:hadoop-common") From 30b9e3861bbba51d8d37cd78085ca4af69c2e355 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Fri, 11 Mar 2022 19:32:01 +0400 Subject: [PATCH 02/31] [BEAM-14048] Fix checkstyle --- .../beam/sdk/io/cdap}/ConfigWrapper.java | 0 .../cdap}/PluginConfigInstantiationUtils.java | 0 .../apache/beam/sdk/io/cdap/package-info.java | 24 +++++++++++++++++++ .../beam/sdk/io/cdap}/ConfigWrapperTest.java | 2 +- 4 files changed, 25 insertions(+), 1 deletion(-) rename sdks/java/io/cdap/src/main/java/{org.apache.beam.sdk.io.cdap => org/apache/beam/sdk/io/cdap}/ConfigWrapper.java (100%) rename sdks/java/io/cdap/src/main/java/{org.apache.beam.sdk.io.cdap => org/apache/beam/sdk/io/cdap}/PluginConfigInstantiationUtils.java (100%) create mode 100644 sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/package-info.java rename sdks/java/io/cdap/src/test/java/{org.apache.beam.sdk.io.cdap => org/apache/beam/sdk/io/cdap}/ConfigWrapperTest.java (98%) diff --git a/sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/ConfigWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java similarity index 100% rename from sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/ConfigWrapper.java rename to sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java diff --git a/sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java similarity index 100% rename from sdks/java/io/cdap/src/main/java/org.apache.beam.sdk.io.cdap/PluginConfigInstantiationUtils.java rename to sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/package-info.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/package-info.java new file mode 100644 index 000000000000..862b13a856f7 --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/package-info.java @@ -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; diff --git a/sdks/java/io/cdap/src/test/java/org.apache.beam.sdk.io.cdap/ConfigWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java similarity index 98% rename from sdks/java/io/cdap/src/test/java/org.apache.beam.sdk.io.cdap/ConfigWrapperTest.java rename to sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java index 82afa041ad7f..9aea711c40ec 100644 --- a/sdks/java/io/cdap/src/test/java/org.apache.beam.sdk.io.cdap/ConfigWrapperTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java @@ -25,12 +25,12 @@ import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; import java.io.File; import java.util.Map; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; /** Test class for {@link ConfigWrapper}. */ @RunWith(JUnit4.class) From 9e5e6d6ac61ee71d716624d95f97a3462cf0711e Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Sat, 12 Mar 2022 00:21:47 +0400 Subject: [PATCH 03/31] [BEAM-14048] Fix warnings --- .../org/apache/beam/sdk/io/cdap/ConfigWrapper.java | 2 +- .../sdk/io/cdap/PluginConfigInstantiationUtils.java | 13 ++++++++----- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java index b32f6c2241e4..e5080f97b18f 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java @@ -32,7 +32,7 @@ public class ConfigWrapper { private static final Logger LOG = LoggerFactory.getLogger(ConfigWrapper.class); - private Map paramsMap; + private Map paramsMap = null; private final Class configClass; public ConfigWrapper(Class configClass) { diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index f60dab39e193..29d5b35e05f8 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -28,12 +28,14 @@ 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") public class PluginConfigInstantiationUtils { private static final Logger LOG = LoggerFactory.getLogger(PluginConfigInstantiationUtils.class); @@ -51,7 +53,7 @@ public static T getPluginConfig( } List allFields = new ArrayList<>(); Class currClass = configClass; - while (!currClass.equals(Object.class)) { + while (!Object.class.equals(currClass)) { allFields.addAll( Arrays.stream(currClass.getDeclaredFields()) .filter( @@ -66,8 +68,9 @@ public static T getPluginConfig( Class fieldType = field.getType(); - String fieldName = field.getDeclaredAnnotation(Name.class).value(); - Object fieldValue = params.get(fieldName); + Name declaredAnnotation = field.getDeclaredAnnotation(Name.class); + Object fieldValue = + declaredAnnotation != null ? params.get(declaredAnnotation.value()) : null; if (fieldValue != null && fieldType.equals(fieldValue.getClass())) { try { @@ -81,7 +84,7 @@ public static T getPluginConfig( } /** @return empty {@link Object} of {@param tClass} */ - private static T getEmptyObjectOf(Class tClass) { + private static @Nullable T getEmptyObjectOf(Class tClass) { for (Constructor constructor : tClass.getDeclaredConstructors()) { constructor.setAccessible(true); Class[] parameterTypes = constructor.getParameterTypes(); @@ -99,7 +102,7 @@ private static T getEmptyObjectOf(Class tClass) { } /** @return default value for given {@param tClass} */ - private static Object getDefaultValue(Class tClass) { + private static @Nullable Object getDefaultValue(Class tClass) { if (Boolean.TYPE.equals(tClass)) { return false; } From 2d414f457c6ae9c9992f81ae4e7e85fe3321ef09 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 15 Mar 2022 15:54:03 +0400 Subject: [PATCH 04/31] [BEAM-14048] Fix warnings --- .../beam/sdk/io/cdap/ConfigWrapper.java | 3 +- .../cdap/PluginConfigInstantiationUtils.java | 32 ++++++++++--------- 2 files changed, 19 insertions(+), 16 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java index e5080f97b18f..b30f4f83b723 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; +import javax.annotation.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,7 +33,7 @@ public class ConfigWrapper { private static final Logger LOG = LoggerFactory.getLogger(ConfigWrapper.class); - private Map paramsMap = null; + @Nullable private Map paramsMap = null; private final Class configClass; public ConfigWrapper(Class configClass) { diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index 29d5b35e05f8..ef2f84bdd9d0 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -45,7 +45,7 @@ public class PluginConfigInstantiationUtils { * boxed primitive * @return Config object for given map of arguments and configuration class */ - public static T getPluginConfig( + public static @Nullable T getPluginConfig( Map params, Class configClass) { // Validate configClass if (configClass == null || configClass.isPrimitive() || configClass.isArray()) { @@ -53,7 +53,7 @@ public static T getPluginConfig( } List allFields = new ArrayList<>(); Class currClass = configClass; - while (!Object.class.equals(currClass)) { + while (currClass != null && !currClass.equals(Object.class)) { allFields.addAll( Arrays.stream(currClass.getDeclaredFields()) .filter( @@ -63,20 +63,22 @@ public static T getPluginConfig( } T config = getEmptyObjectOf(configClass); - for (Field field : allFields) { - field.setAccessible(true); + if (config != null) { + for (Field field : allFields) { + field.setAccessible(true); - Class fieldType = field.getType(); + Class fieldType = field.getType(); - Name declaredAnnotation = field.getDeclaredAnnotation(Name.class); - Object fieldValue = - declaredAnnotation != null ? params.get(declaredAnnotation.value()) : null; + 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); + if (fieldValue != null && fieldType.equals(fieldValue.getClass())) { + try { + field.set(config, fieldValue); + } catch (IllegalAccessException e) { + LOG.error("Can not set a field", e); + } } } } @@ -102,7 +104,7 @@ public static T getPluginConfig( } /** @return default value for given {@param tClass} */ - private static @Nullable Object getDefaultValue(Class tClass) { + private static Object getDefaultValue(Class tClass) { if (Boolean.TYPE.equals(tClass)) { return false; } @@ -127,6 +129,6 @@ public static T getPluginConfig( if (Long.TYPE.equals(tClass)) { return Long.MIN_VALUE; } - return null; + return new Object(); } } From d35acb8ee92752a10802b7d9865a48bfb0644551 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 15 Mar 2022 17:19:46 +0400 Subject: [PATCH 05/31] [BEAM-14048] Fix warning --- .../main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java index b30f4f83b723..aa65c5c5781b 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java @@ -24,6 +24,7 @@ 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; @@ -76,7 +77,7 @@ public T build() { return PluginConfigInstantiationUtils.getPluginConfig(getParamsMap(), configClass); } - private Map getParamsMap() { + private @Nonnull Map getParamsMap() { if (paramsMap == null) { paramsMap = new HashMap<>(); } From b7e0fca7df9c691e97caeb517b2a0fd6c58f6113 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 15 Mar 2022 21:01:20 +0400 Subject: [PATCH 06/31] [BEAM-14048] Fix warning --- .../main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java | 2 +- .../java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java index aa65c5c5781b..55fb3091e2be 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java @@ -73,7 +73,7 @@ public ConfigWrapper setParam(String paramName, Object param) { return this; } - public T build() { + public @Nullable T build() { return PluginConfigInstantiationUtils.getPluginConfig(getParamsMap(), configClass); } diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java index 9aea711c40ec..f356c38ddfaf 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.cdap; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; import io.cdap.plugin.salesforce.SalesforceConstants; @@ -73,6 +74,7 @@ public void testBuildingPluginConfigFromParamsMap() { .withParams(TEST_SALESFORCE_PARAMS_MAP) .setParam("referenceName", newReferenceName) .build(); + assertNotNull(firstConfig); validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, firstConfig); assertEquals(newReferenceName, firstConfig.referenceName); } catch (Exception e) { @@ -90,6 +92,7 @@ public void testBuildingPluginConfigFromJsonFile() { .fromJsonFile(new File(SALESFORCE_TEST_PARAMS_JSON)) .setParam(REFERENCE_NAME_PARAM_NAME, newReferenceName) .build(); + assertNotNull(secondConfig); validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, secondConfig); assertEquals(newReferenceName, secondConfig.referenceName); } catch (Exception e) { @@ -107,6 +110,7 @@ public void testBuildingPluginConfigFromJsonString() { .fromJsonString(TEST_SALESFORCE_PARAMS_JSON_STRING) .setParam(REFERENCE_NAME_PARAM_NAME, newReferenceName) .build(); + assertNotNull(secondConfig); validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, secondConfig); assertEquals(newReferenceName, secondConfig.referenceName); } catch (Exception e) { From 32523952b3c322e1cbbd9a41cbd01eaa2e89a1fc Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 15 Mar 2022 22:05:07 +0400 Subject: [PATCH 07/31] [BEAM-14048] Remove unused dependencies --- sdks/java/io/cdap/build.gradle | 17 ++--------------- .../io/cdap/PluginConfigInstantiationUtils.java | 12 ++++++------ 2 files changed, 8 insertions(+), 21 deletions(-) diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 80143c428488..d224d58fa086 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -45,27 +45,14 @@ allprojects { dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(":sdks:java:io:hadoop-common") - implementation project(":sdks:java:io:hadoop-format") - implementation "org.apache.hadoop:hadoop-mapred:0.22.0" - implementation library.java.vendored_guava_26_0_jre implementation library.java.slf4j_api - implementation "org.apache.commons:commons-lang3:3.9" - implementation "org.apache.httpcomponents:httpclient:4.5.13" - implementation "org.apache.httpcomponents:httpcore:4.4.13" implementation "io.cdap.cdap:cdap-api:6.5.1" - implementation "io.cdap.cdap:cdap-formats:6.5.1" - implementation "io.cdap.cdap:cdap-etl-api:6.5.1" - implementation "com.github.data-integrations:salesforce:v1.3.9" - implementation 'com.google.http-client:google-http-client-gson:1.32.1' - implementation 'com.github.rholder:guava-retrying:2.0.0' - implementation 'commons-validator:commons-validator:1.7' + implementation "com.fasterxml.jackson.core:jackson-core:2.13.0" + implementation "com.fasterxml.jackson.core:jackson-databind:2.13.0" 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(":sdks:java:io:hadoop-common") - testImplementation project(":sdks:java:io:hadoop-format") testImplementation project(path: ":examples:java", configuration: "testRuntimeMigration") testImplementation project(path: ":runners:direct-java", configuration: "shadow") testImplementation project(":sdks:java:io:google-cloud-platform") diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index ef2f84bdd9d0..19306919b307 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -90,10 +90,10 @@ public class PluginConfigInstantiationUtils { for (Constructor constructor : tClass.getDeclaredConstructors()) { constructor.setAccessible(true); Class[] parameterTypes = constructor.getParameterTypes(); - Object[] parameters = - Arrays.stream(parameterTypes) - .map(PluginConfigInstantiationUtils::getDefaultValue) - .toArray(); + 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) { @@ -104,7 +104,7 @@ public class PluginConfigInstantiationUtils { } /** @return default value for given {@param tClass} */ - private static Object getDefaultValue(Class tClass) { + private static @Nullable Object getDefaultValue(Class tClass) { if (Boolean.TYPE.equals(tClass)) { return false; } @@ -129,6 +129,6 @@ private static Object getDefaultValue(Class tClass) { if (Long.TYPE.equals(tClass)) { return Long.MIN_VALUE; } - return new Object(); + return null; } } From f9e56753dbbefdc3bececb93cba9feaa13b2202f Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 15 Mar 2022 22:27:27 +0400 Subject: [PATCH 08/31] [BEAM-14048] Add needed dependencies --- sdks/java/io/cdap/build.gradle | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index d224d58fa086..d9ea334410c3 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -45,10 +45,14 @@ allprojects { dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.vendored_guava_26_0_jre implementation library.java.slf4j_api + implementation "org.apache.commons:commons-lang3:3.9" 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" + implementation "io.cdap.cdap:cdap-formats:6.5.1" + implementation "io.cdap.cdap:cdap-etl-api:6.5.1" + implementation "com.github.data-integrations:salesforce:v1.3.9" + implementation 'com.google.http-client:google-http-client-gson:1.32.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") From 65a5442a842ac77b9a5212a4d6d57c2f00e8a4b1 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 15 Mar 2022 23:39:23 +0400 Subject: [PATCH 09/31] [BEAM-14048] Fix spotless --- .../beam/sdk/io/cdap/PluginConfigInstantiationUtils.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index 19306919b307..63d7ede09279 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -91,9 +91,7 @@ public class PluginConfigInstantiationUtils { constructor.setAccessible(true); Class[] parameterTypes = constructor.getParameterTypes(); Object[] parameters = new Object[parameterTypes.length]; - for (int i = 0; i < parameterTypes.length; i++) { - parameters[i] = getDefaultValue(parameterTypes[i]); - } + Arrays.stream(parameterTypes).map(PluginConfigInstantiationUtils::getDefaultValue).toArray(); try { return (T) constructor.newInstance(parameters); } catch (InstantiationException | IllegalAccessException | InvocationTargetException e) { @@ -104,7 +102,7 @@ public class PluginConfigInstantiationUtils { } /** @return default value for given {@param tClass} */ - private static @Nullable Object getDefaultValue(Class tClass) { + private static @Nullable Object getDefaultValue(@Nullable Class tClass) { if (Boolean.TYPE.equals(tClass)) { return false; } From 00aef640c5c8b7b671a65a057a924cbcbfac924e Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 15 Mar 2022 23:40:56 +0400 Subject: [PATCH 10/31] [BEAM-14048] Fix typo --- .../beam/sdk/io/cdap/PluginConfigInstantiationUtils.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index 63d7ede09279..858caf845ae4 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -90,8 +90,10 @@ public class PluginConfigInstantiationUtils { for (Constructor constructor : tClass.getDeclaredConstructors()) { constructor.setAccessible(true); Class[] parameterTypes = constructor.getParameterTypes(); - Object[] parameters = new Object[parameterTypes.length]; - Arrays.stream(parameterTypes).map(PluginConfigInstantiationUtils::getDefaultValue).toArray(); + Object[] parameters = + Arrays.stream(parameterTypes) + .map(PluginConfigInstantiationUtils::getDefaultValue) + .toArray(); try { return (T) constructor.newInstance(parameters); } catch (InstantiationException | IllegalAccessException | InvocationTargetException e) { From d10b84b926c0e4eda25714944a4f8ac30683849c Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Mar 2022 11:41:06 +0400 Subject: [PATCH 11/31] [BEAM-14048] Use fori instead of stream --- .../beam/sdk/io/cdap/PluginConfigInstantiationUtils.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index 858caf845ae4..f56addc3bbb6 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -90,10 +90,10 @@ public class PluginConfigInstantiationUtils { for (Constructor constructor : tClass.getDeclaredConstructors()) { constructor.setAccessible(true); Class[] parameterTypes = constructor.getParameterTypes(); - Object[] parameters = - Arrays.stream(parameterTypes) - .map(PluginConfigInstantiationUtils::getDefaultValue) - .toArray(); + 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) { From 2f9686dd4bae48fd90213d9cce0f95bb94d9a380 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Mar 2022 12:24:46 +0400 Subject: [PATCH 12/31] [BEAM-14048] Suppress warning --- .../apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index f56addc3bbb6..98b7fcc5d28b 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -35,7 +35,7 @@ /** * Class for getting any filled {@link io.cdap.cdap.api.plugin.PluginConfig} configuration object. */ -@SuppressWarnings("unchecked") +@SuppressWarnings({"unchecked", "assignment.type.incompatible"}) public class PluginConfigInstantiationUtils { private static final Logger LOG = LoggerFactory.getLogger(PluginConfigInstantiationUtils.class); From 119d2ccd726dee671c83646a9bc5011653542945 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Mar 2022 12:56:37 +0400 Subject: [PATCH 13/31] [BEAM-14048] Add used undeclared artifacts --- sdks/java/io/cdap/build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index d9ea334410c3..704da50c1c73 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -53,6 +53,8 @@ dependencies { implementation "io.cdap.cdap:cdap-etl-api:6.5.1" implementation "com.github.data-integrations:salesforce:v1.3.9" implementation 'com.google.http-client:google-http-client-gson:1.32.1' + implementation "com.fasterxml.jackson.core:jackson-core:2.13.0" + implementation "com.fasterxml.jackson.core:jackson-databind:2.13.0" 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") From 1f63d659a3b9ba91dfb5619d2d17ab43388743c5 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Mar 2022 14:02:30 +0400 Subject: [PATCH 14/31] [BEAM-14048] Change dependencies to test --- sdks/java/io/cdap/build.gradle | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 704da50c1c73..37865a27ebe6 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -45,16 +45,13 @@ allprojects { dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation library.java.vendored_guava_26_0_jre implementation library.java.slf4j_api - implementation "org.apache.commons:commons-lang3:3.9" implementation "io.cdap.cdap:cdap-api:6.5.1" - implementation "io.cdap.cdap:cdap-formats:6.5.1" - implementation "io.cdap.cdap:cdap-etl-api:6.5.1" - implementation "com.github.data-integrations:salesforce:v1.3.9" - implementation 'com.google.http-client:google-http-client-gson:1.32.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") From 62072241883711d0764236ded6cb558beb9a45a8 Mon Sep 17 00:00:00 2001 From: "igor.krasavin" Date: Wed, 16 Mar 2022 16:28:31 +0300 Subject: [PATCH 15/31] Add context. --- .../sdk/io/cdap/context/BatchContextImpl.java | 224 ++++++++++++++++++ .../io/cdap/context/BatchSinkContextImpl.java | 41 ++++ .../cdap/context/BatchSourceContextImpl.java | 41 ++++ .../cdap/context/FailureCollectorWrapper.java | 55 +++++ .../context/StreamingSourceContextImpl.java | 35 +++ .../io/cdap/context/BatchContextImplTest.java | 59 +++++ .../context/FailureCollectorWrapperTest.java | 93 ++++++++ 7 files changed, 548 insertions(+) create mode 100644 sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java create mode 100644 sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java create mode 100644 sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java create mode 100644 sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java create mode 100644 sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java create mode 100644 sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java create mode 100644 sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java new file mode 100644 index 000000000000..6cdeaa8367e3 --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java @@ -0,0 +1,224 @@ +/* + * 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.context; + +import io.cdap.cdap.api.data.DatasetInstantiationException; +import io.cdap.cdap.api.data.schema.Schema; +import io.cdap.cdap.api.dataset.Dataset; +import io.cdap.cdap.api.dataset.DatasetManagementException; +import io.cdap.cdap.api.dataset.DatasetProperties; +import io.cdap.cdap.api.metadata.Metadata; +import io.cdap.cdap.api.metadata.MetadataEntity; +import io.cdap.cdap.api.metadata.MetadataException; +import io.cdap.cdap.api.metadata.MetadataScope; +import io.cdap.cdap.api.plugin.PluginProperties; +import io.cdap.cdap.etl.api.FailureCollector; +import io.cdap.cdap.etl.api.Lookup; +import io.cdap.cdap.etl.api.StageMetrics; +import io.cdap.cdap.etl.api.action.SettableArguments; +import io.cdap.cdap.etl.api.batch.BatchContext; +import io.cdap.cdap.etl.api.lineage.field.FieldOperation; +import java.net.URL; +import java.sql.Timestamp; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; + +/** + * Class OperationContext is a common class for Batch, Sink and Stream CDAP wrapper classes that use + * it to provide common details. + */ +@SuppressWarnings("TypeParameterUnusedInFormals") +public class BatchContextImpl implements BatchContext { + + private final FailureCollectorWrapper failureCollector = new FailureCollectorWrapper(); + + private final Timestamp startTime = new Timestamp(System.currentTimeMillis()); + + @Override + public String getStageName() { + return null; + } + + @Override + public String getNamespace() { + return null; + } + + @Override + public String getPipelineName() { + return null; + } + + @Override + public long getLogicalStartTime() { + return this.startTime.getTime(); + } + + @Override + public StageMetrics getMetrics() { + return null; + } + + @Override + public PluginProperties getPluginProperties() { + return null; + } + + @Override + public PluginProperties getPluginProperties(String pluginId) { + return null; + } + + @Override + public Class loadPluginClass(String pluginId) { + return null; + } + + @Override + public T newPluginInstance(String pluginId) throws InstantiationException { + return null; + } + + @Nullable + @Override + public Schema getInputSchema() { + return null; + } + + @Override + public Map getInputSchemas() { + return null; + } + + @Nullable + @Override + public Schema getOutputSchema() { + return null; + } + + @Override + public Map getOutputPortSchemas() { + return null; + } + + @Override + public void createDataset(String datasetName, String typeName, DatasetProperties properties) + throws DatasetManagementException {} + + @Override + public boolean datasetExists(String datasetName) throws DatasetManagementException { + return false; + } + + @Override + public SettableArguments getArguments() { + return null; + } + + @Override + public FailureCollector getFailureCollector() { + return this.failureCollector; + } + + @Nullable + @Override + public URL getServiceURL(String applicationId, String serviceId) { + return null; + } + + @Nullable + @Override + public URL getServiceURL(String serviceId) { + return null; + } + + @Override + public Map getMetadata(MetadataEntity metadataEntity) + throws MetadataException { + return null; + } + + @Override + public Metadata getMetadata(MetadataScope scope, MetadataEntity metadataEntity) + throws MetadataException { + return null; + } + + @Override + public void addProperties(MetadataEntity metadataEntity, Map properties) {} + + @Override + public void addTags(MetadataEntity metadataEntity, String... tags) {} + + @Override + public void addTags(MetadataEntity metadataEntity, Iterable tags) {} + + @Override + public void removeMetadata(MetadataEntity metadataEntity) {} + + @Override + public void removeProperties(MetadataEntity metadataEntity) {} + + @Override + public void removeProperties(MetadataEntity metadataEntity, String... keys) {} + + @Override + public void removeTags(MetadataEntity metadataEntity) {} + + @Override + public void removeTags(MetadataEntity metadataEntity, String... tags) {} + + @Override + public void record(List fieldOperations) {} + + @Override + public T getDataset(String name) throws DatasetInstantiationException { + return null; + } + + @Override + public T getDataset(String namespace, String name) + throws DatasetInstantiationException { + return null; + } + + @Override + public T getDataset(String name, Map arguments) + throws DatasetInstantiationException { + return null; + } + + @Override + public T getDataset( + String namespace, String name, Map arguments) + throws DatasetInstantiationException { + return null; + } + + @Override + public void releaseDataset(Dataset dataset) {} + + @Override + public void discardDataset(Dataset dataset) {} + + @Override + public Lookup provide(String table, Map arguments) { + return null; + } +} diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java new file mode 100644 index 000000000000..9bdcb78b3d60 --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java @@ -0,0 +1,41 @@ +/* + * 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.context; + +import io.cdap.cdap.api.data.batch.Output; +import io.cdap.cdap.etl.api.batch.BatchSinkContext; + +/** + * Class BatchSinkContextWrapper is a class for creating context object of different CDAP classes + * with batch sink type. + */ +public class BatchSinkContextImpl extends BatchContextImpl implements BatchSinkContext { + + @Override + public void addOutput(Output output) {} + + @Override + public boolean isPreviewEnabled() { + return false; + } + + @Override + public String getStageName() { + return null; + } +} diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java new file mode 100644 index 000000000000..0fed016d2bd6 --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java @@ -0,0 +1,41 @@ +/* + * 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.context; + +import io.cdap.cdap.api.data.batch.Input; +import io.cdap.cdap.etl.api.batch.BatchSourceContext; + +/** + * Class BatchSourceContextWrapper is a class for creating context object of different CDAP classes + * with batch source type. + */ +public class BatchSourceContextImpl extends BatchContextImpl implements BatchSourceContext { + + @Override + public void setInput(Input input) {} + + @Override + public boolean isPreviewEnabled() { + return false; + } + + @Override + public int getMaxPreviewRecords() { + return 0; + } +} diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java new file mode 100644 index 000000000000..d697909d02ef --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java @@ -0,0 +1,55 @@ +/* + * 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.context; + +import io.cdap.cdap.etl.api.FailureCollector; +import io.cdap.cdap.etl.api.validation.ValidationException; +import io.cdap.cdap.etl.api.validation.ValidationFailure; +import java.util.ArrayList; +import javax.annotation.Nullable; + +/** Class FailureCollectorWrapper is a class for collecting ValidationFailure. */ +public class FailureCollectorWrapper implements FailureCollector { + private ArrayList failuresCollection; + + public FailureCollectorWrapper() { + this.failuresCollection = new ArrayList<>(); + } + + @Override + public ValidationFailure addFailure(String message, @Nullable String correctiveAction) { + ValidationFailure validationFailure = new ValidationFailure(message, correctiveAction); + failuresCollection.add(validationFailure); + + return validationFailure; + } + + @Override + public ValidationException getOrThrowException() throws ValidationException { + if (failuresCollection.isEmpty()) { + return new ValidationException(this.failuresCollection); + } + + throw new ValidationException(this.failuresCollection); + } + + @Override + public ArrayList getValidationFailures() { + return this.failuresCollection; + } +} diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java new file mode 100644 index 000000000000..b357a9291041 --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java @@ -0,0 +1,35 @@ +/* + * 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.context; + +import io.cdap.cdap.api.data.schema.Schema; +import io.cdap.cdap.api.dataset.DatasetManagementException; +import io.cdap.cdap.etl.api.streaming.StreamingSourceContext; +import javax.annotation.Nullable; +import org.apache.tephra.TransactionFailureException; + +/** + * Class StreamingSourceContextWrapper is a class for creating context object of different CDAP + * classes with stream source type. + */ +public class StreamingSourceContextImpl extends BatchContextImpl implements StreamingSourceContext { + + @Override + public void registerLineage(String referenceName, @Nullable Schema schema) + throws DatasetManagementException, TransactionFailureException {} +} diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java new file mode 100644 index 000000000000..6e58d832e8dd --- /dev/null +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java @@ -0,0 +1,59 @@ +/* + * 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.context; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import io.cdap.cdap.etl.api.FailureCollector; +import io.cdap.cdap.etl.api.validation.ValidationException; +import java.sql.Timestamp; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test class for {@link BatchContextImpl}. */ +@RunWith(JUnit4.class) +public class BatchContextImplTest { + + @Test + public void getLogicalStartTime() { + /** arrange */ + Timestamp expectedStartTime = new Timestamp(System.currentTimeMillis()); + BatchContextImpl context = new BatchContextImpl(); + + /** act */ + long actualStartTime = context.getLogicalStartTime(); + + /** assert */ + assertTrue((expectedStartTime.getTime() - actualStartTime) <= 100); + } + + @Test + public void getFailureCollector() { + /** arrange */ + BatchContextImpl context = new BatchContextImpl(); + + /** act */ + FailureCollector failureCollector = context.getFailureCollector(); + + /** assert */ + ValidationException validationException = failureCollector.getOrThrowException(); + assertEquals(0, validationException.getFailures().size()); + } +} diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java new file mode 100644 index 000000000000..0e35c8a06a59 --- /dev/null +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java @@ -0,0 +1,93 @@ +/* + * 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.context; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import io.cdap.cdap.etl.api.validation.ValidationException; +import io.cdap.cdap.etl.api.validation.ValidationFailure; +import java.util.ArrayList; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test class for {@link FailureCollectorWrapper}. */ +@RunWith(JUnit4.class) +public class FailureCollectorWrapperTest { + + @Test + public void addFailure() { + /** arrange */ + FailureCollectorWrapper failureCollectorWrapper = new FailureCollectorWrapper(); + + /** act */ + RuntimeException error = new RuntimeException("An error has occurred"); + failureCollectorWrapper.addFailure(error.getMessage(), null); + + /** assert */ + assertThrows(ValidationException.class, () -> failureCollectorWrapper.getOrThrowException()); + } + + @Test + public void getOrThrowException() { + /** arrange */ + FailureCollectorWrapper failureCollectorWrapper = new FailureCollectorWrapper(); + String errorMessage = "An error has occurred"; + String expectedMessage = "Errors were encountered during validation. An error has occurred"; + + FailureCollectorWrapper emptyFailureCollectorWrapper = new FailureCollectorWrapper(); + + RuntimeException error = new RuntimeException(errorMessage); + failureCollectorWrapper.addFailure(error.getMessage(), null); + + /** act && assert */ + ValidationException e = + assertThrows( + ValidationException.class, () -> failureCollectorWrapper.getOrThrowException()); + assertEquals(expectedMessage, e.getMessage()); + + // A case when return ValidationException with empty collector + ArrayList exceptionCollector = + emptyFailureCollectorWrapper.getValidationFailures(); + assertEquals(0, exceptionCollector.size()); + } + + @Test + public void getValidationFailures() { + /** arrange */ + FailureCollectorWrapper failureCollectorWrapper = new FailureCollectorWrapper(); + String errorMessage = "An error has occurred"; + + FailureCollectorWrapper emptyFailureCollectorWrapper = new FailureCollectorWrapper(); + + RuntimeException error = new RuntimeException(errorMessage); + failureCollectorWrapper.addFailure(error.getMessage(), null); + + /** act */ + ArrayList exceptionCollector = + failureCollectorWrapper.getValidationFailures(); + ArrayList emptyExceptionCollector = + emptyFailureCollectorWrapper.getValidationFailures(); + + /** assert */ + assertEquals(1, exceptionCollector.size()); + assertEquals(errorMessage, exceptionCollector.get(0).getMessage()); + assertEquals(0, emptyExceptionCollector.size()); + } +} From 1dbd4d6b6336ce2c470bba235125f12e9997439a Mon Sep 17 00:00:00 2001 From: "igor.krasavin" Date: Wed, 16 Mar 2022 17:32:54 +0300 Subject: [PATCH 16/31] Fix dependencies issue --- sdks/java/io/cdap/build.gradle | 3 ++- .../sdk/io/cdap/context/package-info.java | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) create mode 100644 sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/package-info.java diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 37865a27ebe6..30e2ba7d6eb3 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -49,9 +49,10 @@ dependencies { 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" + implementation "io.cdap.cdap:cdap-etl-api:6.5.1" + implementation "io.cdap.cdap:cdap-etl-api-spark:6.2.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") diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/package-info.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/package-info.java new file mode 100644 index 000000000000..f6548ccdf932 --- /dev/null +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/package-info.java @@ -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. + */ + +/** Context for CDAP classes. */ +@Experimental(Kind.SOURCE_SINK) +package org.apache.beam.sdk.io.cdap.context; + +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; From 0777205e41e1032c9a716bd579dbd5382e2bdb85 Mon Sep 17 00:00:00 2001 From: "igor.krasavin" Date: Thu, 17 Mar 2022 19:26:25 +0300 Subject: [PATCH 17/31] Add null annotation --- .../sdk/io/cdap/context/BatchContextImpl.java | 44 +++++++++---------- .../io/cdap/context/BatchSinkContextImpl.java | 3 +- 2 files changed, 22 insertions(+), 25 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java index 6cdeaa8367e3..89cd1c443bd2 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java @@ -51,17 +51,17 @@ public class BatchContextImpl implements BatchContext { private final Timestamp startTime = new Timestamp(System.currentTimeMillis()); @Override - public String getStageName() { + public @Nullable String getStageName() { return null; } @Override - public String getNamespace() { + public @Nullable String getNamespace() { return null; } @Override - public String getPipelineName() { + public @Nullable String getPipelineName() { return null; } @@ -76,44 +76,42 @@ public StageMetrics getMetrics() { } @Override - public PluginProperties getPluginProperties() { + public @Nullable PluginProperties getPluginProperties() { return null; } @Override - public PluginProperties getPluginProperties(String pluginId) { + public @Nullable PluginProperties getPluginProperties(String pluginId) { return null; } @Override - public Class loadPluginClass(String pluginId) { + public @Nullable Class loadPluginClass(String pluginId) { return null; } @Override - public T newPluginInstance(String pluginId) throws InstantiationException { + public @Nullable T newPluginInstance(String pluginId) throws InstantiationException { return null; } - @Nullable @Override - public Schema getInputSchema() { + public @Nullable Schema getInputSchema() { return null; } @Override - public Map getInputSchemas() { + public @Nullable Map getInputSchemas() { return null; } - @Nullable @Override - public Schema getOutputSchema() { + public @Nullable Schema getOutputSchema() { return null; } @Override - public Map getOutputPortSchemas() { + public @Nullable Map getOutputPortSchemas() { return null; } @@ -127,7 +125,7 @@ public boolean datasetExists(String datasetName) throws DatasetManagementExcepti } @Override - public SettableArguments getArguments() { + public @Nullable SettableArguments getArguments() { return null; } @@ -136,26 +134,24 @@ public FailureCollector getFailureCollector() { return this.failureCollector; } - @Nullable @Override - public URL getServiceURL(String applicationId, String serviceId) { + public @Nullable URL getServiceURL(String applicationId, String serviceId) { return null; } - @Nullable @Override - public URL getServiceURL(String serviceId) { + public @Nullable URL getServiceURL(String serviceId) { return null; } @Override - public Map getMetadata(MetadataEntity metadataEntity) + public @Nullable Map getMetadata(MetadataEntity metadataEntity) throws MetadataException { return null; } @Override - public Metadata getMetadata(MetadataScope scope, MetadataEntity metadataEntity) + public @Nullable Metadata getMetadata(MetadataScope scope, MetadataEntity metadataEntity) throws MetadataException { return null; } @@ -193,19 +189,19 @@ public T getDataset(String name) throws DatasetInstantiation } @Override - public T getDataset(String namespace, String name) + public @Nullable T getDataset(String namespace, String name) throws DatasetInstantiationException { return null; } @Override - public T getDataset(String name, Map arguments) + public @Nullable T getDataset(String name, Map arguments) throws DatasetInstantiationException { return null; } @Override - public T getDataset( + public @Nullable T getDataset( String namespace, String name, Map arguments) throws DatasetInstantiationException { return null; @@ -218,7 +214,7 @@ public void releaseDataset(Dataset dataset) {} public void discardDataset(Dataset dataset) {} @Override - public Lookup provide(String table, Map arguments) { + public @Nullable Lookup provide(String table, Map arguments) { return null; } } diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java index 9bdcb78b3d60..be34bd2f3139 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java @@ -19,6 +19,7 @@ import io.cdap.cdap.api.data.batch.Output; import io.cdap.cdap.etl.api.batch.BatchSinkContext; +import javax.annotation.Nullable; /** * Class BatchSinkContextWrapper is a class for creating context object of different CDAP classes @@ -35,7 +36,7 @@ public boolean isPreviewEnabled() { } @Override - public String getStageName() { + public @Nullable String getStageName() { return null; } } From 0645c7b078c52b53fdf4a5abb307588298ebd78b Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 22 Mar 2022 15:43:54 +0400 Subject: [PATCH 18/31] [BEAM-14048] Refactoring --- .../beam/gradle/BeamModulePlugin.groovy | 3 + sdks/java/io/cdap/build.gradle | 25 +---- .../beam/sdk/io/cdap/ConfigWrapper.java | 6 +- .../cdap/PluginConfigInstantiationUtils.java | 24 ++-- .../beam/sdk/io/cdap/ConfigWrapperTest.java | 24 ++-- .../PluginConfigInstantiationUtilsTest.java | 105 ++++++++++++++++++ 6 files changed, 143 insertions(+), 44 deletions(-) create mode 100644 sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtilsTest.java diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 08b3dca52a3d..26348720c9d3 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -451,6 +451,7 @@ class BeamModulePlugin implements Plugin { def aws_java_sdk_version = "1.12.135" def aws_java_sdk2_version = "2.17.106" def cassandra_driver_version = "3.10.2" + def cdap_version = "6.5.1" def checkerframework_version = "3.10.0" def classgraph_version = "4.8.104" def errorprone_version = "2.10.0" @@ -535,6 +536,8 @@ class BeamModulePlugin implements Plugin { bigdataoss_util : "com.google.cloud.bigdataoss:util:$google_cloud_bigdataoss_version", cassandra_driver_core : "com.datastax.cassandra:cassandra-driver-core:$cassandra_driver_version", cassandra_driver_mapping : "com.datastax.cassandra:cassandra-driver-mapping:$cassandra_driver_version", + cdap_api : "io.cdap.cdap:cdap-api:$cdap_version", + cdap_etl_api : "io.cdap.cdap:cdap-etl-api:$cdap_version", checker_qual : "org.checkerframework:checker-qual:$checkerframework_version", classgraph : "io.github.classgraph:classgraph:$classgraph_version", commons_codec : "commons-codec:commons-codec:1.15", diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 37865a27ebe6..c47fbbb1214f 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -44,26 +44,13 @@ allprojects { } dependencies { - implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.cdap_api + implementation library.java.jackson_core + implementation library.java.jackson_databind 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 + implementation project(path: ":sdks:java:core", configuration: "shadow") 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.cdap_etl_api + testImplementation library.java.vendored_guava_26_0_jre testImplementation library.java.junit - testImplementation library.java.mockito_core - testImplementation library.java.testcontainers_kafka - testImplementation library.java.testcontainers_gcloud } diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java index 55fb3091e2be..9a2124e21b46 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/ConfigWrapper.java @@ -41,24 +41,26 @@ public ConfigWrapper(Class configClass) { this.configClass = configClass; } - public ConfigWrapper fromJsonString(String jsonString) { + public ConfigWrapper fromJsonString(String jsonString) throws IOException { TypeReference> typeRef = new TypeReference>() {}; try { paramsMap = new ObjectMapper().readValue(jsonString, typeRef); } catch (IOException e) { LOG.error("Can not read json string to params map", e); + throw e; } return this; } - public ConfigWrapper fromJsonFile(File jsonFile) { + public ConfigWrapper fromJsonFile(File jsonFile) throws IOException { TypeReference> typeRef = new TypeReference>() {}; try { paramsMap = new ObjectMapper().readValue(jsonFile, typeRef); } catch (IOException e) { LOG.error("Can not read json file to params map", e); + throw e; } return this; } diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index 98b7fcc5d28b..187b859217f2 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -32,24 +32,26 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * Class for getting any filled {@link io.cdap.cdap.api.plugin.PluginConfig} configuration object. - */ +/** Class for getting any filled {@link PluginConfig} configuration object. */ @SuppressWarnings({"unchecked", "assignment.type.incompatible"}) public class PluginConfigInstantiationUtils { private static final Logger LOG = LoggerFactory.getLogger(PluginConfigInstantiationUtils.class); /** + * Method for instantiating {@link PluginConfig} object of specific class {@param configClass}. + * After instantiating, it will go over all {@link Field}s with the {@link Name} annotation and + * set the appropriate parameter values from the {@param params} map for them. + * * @param params map of config fields, where key is the name of the field, value must be String or * boxed primitive * @return Config object for given map of arguments and configuration class */ - public static @Nullable T getPluginConfig( + static @Nullable T getPluginConfig( Map params, Class configClass) { // Validate configClass - if (configClass == null || configClass.isPrimitive() || configClass.isArray()) { - throw new IllegalArgumentException("Config class must be correct!"); + if (configClass == null) { + throw new IllegalArgumentException("Config class must be not null!"); } List allFields = new ArrayList<>(); Class currClass = configClass; @@ -61,7 +63,7 @@ public class PluginConfigInstantiationUtils { .collect(Collectors.toList())); currClass = currClass.getSuperclass(); } - T config = getEmptyObjectOf(configClass); + T config = getEmptyObjectFromDefaultValues(configClass); if (config != null) { for (Field field : allFields) { @@ -77,7 +79,7 @@ public class PluginConfigInstantiationUtils { try { field.set(config, fieldValue); } catch (IllegalAccessException e) { - LOG.error("Can not set a field", e); + LOG.error("Can not set a field with value {}", fieldValue); } } } @@ -86,7 +88,7 @@ public class PluginConfigInstantiationUtils { } /** @return empty {@link Object} of {@param tClass} */ - private static @Nullable T getEmptyObjectOf(Class tClass) { + private static @Nullable T getEmptyObjectFromDefaultValues(Class tClass) { for (Constructor constructor : tClass.getDeclaredConstructors()) { constructor.setAccessible(true); Class[] parameterTypes = constructor.getParameterTypes(); @@ -97,13 +99,13 @@ public class PluginConfigInstantiationUtils { try { return (T) constructor.newInstance(parameters); } catch (InstantiationException | IllegalAccessException | InvocationTargetException e) { - LOG.error("Can not instantiate an empty object", e); + LOG.warn("Can not instantiate an empty object", e); } } return null; } - /** @return default value for given {@param tClass} */ + /** @return default value for given {@param tClass} if it's primitive, otherwise returns null */ private static @Nullable Object getDefaultValue(@Nullable Class tClass) { if (Boolean.TYPE.equals(tClass)) { return false; diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java index f356c38ddfaf..e5b33de83113 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java @@ -69,14 +69,14 @@ public class ConfigWrapperTest { public void testBuildingPluginConfigFromParamsMap() { try { String newReferenceName = "new reference name"; - SalesforceSourceConfig firstConfig = + SalesforceSourceConfig config = new ConfigWrapper<>(SalesforceSourceConfig.class) .withParams(TEST_SALESFORCE_PARAMS_MAP) .setParam("referenceName", newReferenceName) .build(); - assertNotNull(firstConfig); - validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, firstConfig); - assertEquals(newReferenceName, firstConfig.referenceName); + assertNotNull(config); + validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, config); + assertEquals(newReferenceName, config.referenceName); } catch (Exception e) { LOG.error("Error occurred while building the config object", e); fail(); @@ -87,14 +87,14 @@ public void testBuildingPluginConfigFromParamsMap() { public void testBuildingPluginConfigFromJsonFile() { try { String newReferenceName = "new reference name"; - SalesforceSourceConfig secondConfig = + SalesforceSourceConfig config = new ConfigWrapper<>(SalesforceSourceConfig.class) .fromJsonFile(new File(SALESFORCE_TEST_PARAMS_JSON)) .setParam(REFERENCE_NAME_PARAM_NAME, newReferenceName) .build(); - assertNotNull(secondConfig); - validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, secondConfig); - assertEquals(newReferenceName, secondConfig.referenceName); + assertNotNull(config); + validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, config); + assertEquals(newReferenceName, config.referenceName); } catch (Exception e) { LOG.error("Error occurred while building the config object", e); fail(); @@ -105,14 +105,14 @@ public void testBuildingPluginConfigFromJsonFile() { public void testBuildingPluginConfigFromJsonString() { try { String newReferenceName = "new reference name"; - SalesforceSourceConfig secondConfig = + SalesforceSourceConfig config = new ConfigWrapper<>(SalesforceSourceConfig.class) .fromJsonString(TEST_SALESFORCE_PARAMS_JSON_STRING) .setParam(REFERENCE_NAME_PARAM_NAME, newReferenceName) .build(); - assertNotNull(secondConfig); - validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, secondConfig); - assertEquals(newReferenceName, secondConfig.referenceName); + assertNotNull(config); + validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, config); + assertEquals(newReferenceName, config.referenceName); } catch (Exception e) { LOG.error("Error occurred while building the config object", e); fail(); diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtilsTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtilsTest.java new file mode 100644 index 000000000000..394cf45f5600 --- /dev/null +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtilsTest.java @@ -0,0 +1,105 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; + +import io.cdap.plugin.salesforce.SalesforceConstants; +import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceSourceConfig; +import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Test class for {@link PluginConfigInstantiationUtils}. */ +@RunWith(JUnit4.class) +public class PluginConfigInstantiationUtilsTest { + + private static final Logger LOG = + LoggerFactory.getLogger(PluginConfigInstantiationUtilsTest.class); + + private static final ImmutableMap TEST_SALESFORCE_PARAMS_MAP = + ImmutableMap.builder() + .put("sObjectName", "sObject") + .put("datetimeAfter", "datetime") + .put("consumerKey", "key") + .put("consumerSecret", "secret") + .put("username", "user") + .put("password", "password") + .put("loginUrl", "https://www.google.com") + .put("referenceName", "oldReference") + .build(); + + @Test + public void testBuildingPluginConfigFromParamsMap() { + try { + SalesforceSourceConfig config = + PluginConfigInstantiationUtils.getPluginConfig( + TEST_SALESFORCE_PARAMS_MAP, SalesforceSourceConfig.class); + assertNotNull(config); + validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, config); + } catch (Exception e) { + LOG.error("Error occurred while building the config object", e); + fail(); + } + } + + @Test + public void testBuildingPluginConfigFromEmptyParamsMap() { + try { + SalesforceSourceConfig config = + PluginConfigInstantiationUtils.getPluginConfig( + new HashMap<>(), SalesforceSourceConfig.class); + assertNotNull(config); + } catch (Exception e) { + LOG.error("Error occurred while building the config object", e); + fail(); + } + } + + @Test + public void testBuildingPluginConfigFromNullClassFail() { + try { + PluginConfigInstantiationUtils.getPluginConfig(TEST_SALESFORCE_PARAMS_MAP, null); + fail(); + } catch (IllegalArgumentException e) { + assertEquals("Config class must be not null!", e.getMessage()); + } + } + + private static void validateSalesforceConfigObject( + Map params, SalesforceSourceConfig config) { + assertEquals( + params.get(SalesforceSourceConstants.PROPERTY_DATETIME_AFTER), config.getDatetimeAfter()); + assertEquals( + params.get(SalesforceSourceConstants.PROPERTY_SOBJECT_NAME), config.getSObjectName()); + assertEquals(params.get(SalesforceConstants.PROPERTY_CONSUMER_KEY), config.getConsumerKey()); + assertEquals( + params.get(SalesforceConstants.PROPERTY_CONSUMER_SECRET), config.getConsumerSecret()); + assertEquals(params.get(SalesforceConstants.PROPERTY_USERNAME), config.getUsername()); + assertEquals(params.get(SalesforceConstants.PROPERTY_PASSWORD), config.getPassword()); + assertEquals(params.get(SalesforceConstants.PROPERTY_LOGIN_URL), config.getLoginUrl()); + } +} From 72513073f5fe0e63e90bf6b02387bcad451bf32c Mon Sep 17 00:00:00 2001 From: "igor.krasavin" Date: Wed, 23 Mar 2022 13:41:48 +0300 Subject: [PATCH 19/31] Add SuppressWarning. --- .../sdk/io/cdap/context/BatchContextImpl.java | 38 +++++++++---------- .../io/cdap/context/BatchSinkContextImpl.java | 5 --- 2 files changed, 19 insertions(+), 24 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java index 89cd1c443bd2..a1d384061d10 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java @@ -43,7 +43,7 @@ * Class OperationContext is a common class for Batch, Sink and Stream CDAP wrapper classes that use * it to provide common details. */ -@SuppressWarnings("TypeParameterUnusedInFormals") +@SuppressWarnings({"TypeParameterUnusedInFormals", "nullness"}) public class BatchContextImpl implements BatchContext { private final FailureCollectorWrapper failureCollector = new FailureCollectorWrapper(); @@ -51,17 +51,17 @@ public class BatchContextImpl implements BatchContext { private final Timestamp startTime = new Timestamp(System.currentTimeMillis()); @Override - public @Nullable String getStageName() { + public String getStageName() { return null; } @Override - public @Nullable String getNamespace() { + public String getNamespace() { return null; } @Override - public @Nullable String getPipelineName() { + public String getPipelineName() { return null; } @@ -76,27 +76,27 @@ public StageMetrics getMetrics() { } @Override - public @Nullable PluginProperties getPluginProperties() { + public PluginProperties getPluginProperties() { return null; } @Override - public @Nullable PluginProperties getPluginProperties(String pluginId) { + public PluginProperties getPluginProperties(String pluginId) { return null; } @Override - public @Nullable Class loadPluginClass(String pluginId) { + public Class loadPluginClass(String pluginId) { return null; } @Override - public @Nullable T newPluginInstance(String pluginId) throws InstantiationException { + public T newPluginInstance(String pluginId) throws InstantiationException { return null; } @Override - public @Nullable Schema getInputSchema() { + public Schema getInputSchema() { return null; } @@ -111,7 +111,7 @@ public StageMetrics getMetrics() { } @Override - public @Nullable Map getOutputPortSchemas() { + public Map getOutputPortSchemas() { return null; } @@ -125,7 +125,7 @@ public boolean datasetExists(String datasetName) throws DatasetManagementExcepti } @Override - public @Nullable SettableArguments getArguments() { + public SettableArguments getArguments() { return null; } @@ -135,23 +135,23 @@ public FailureCollector getFailureCollector() { } @Override - public @Nullable URL getServiceURL(String applicationId, String serviceId) { + public URL getServiceURL(String applicationId, String serviceId) { return null; } @Override - public @Nullable URL getServiceURL(String serviceId) { + public URL getServiceURL(String serviceId) { return null; } @Override - public @Nullable Map getMetadata(MetadataEntity metadataEntity) + public Map getMetadata(MetadataEntity metadataEntity) throws MetadataException { return null; } @Override - public @Nullable Metadata getMetadata(MetadataScope scope, MetadataEntity metadataEntity) + public Metadata getMetadata(MetadataScope scope, MetadataEntity metadataEntity) throws MetadataException { return null; } @@ -189,19 +189,19 @@ public T getDataset(String name) throws DatasetInstantiation } @Override - public @Nullable T getDataset(String namespace, String name) + public T getDataset(String namespace, String name) throws DatasetInstantiationException { return null; } @Override - public @Nullable T getDataset(String name, Map arguments) + public T getDataset(String name, Map arguments) throws DatasetInstantiationException { return null; } @Override - public @Nullable T getDataset( + public T getDataset( String namespace, String name, Map arguments) throws DatasetInstantiationException { return null; @@ -214,7 +214,7 @@ public void releaseDataset(Dataset dataset) {} public void discardDataset(Dataset dataset) {} @Override - public @Nullable Lookup provide(String table, Map arguments) { + public Lookup provide(String table, Map arguments) { return null; } } diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java index be34bd2f3139..dcdf7ea9f7f1 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java @@ -34,9 +34,4 @@ public void addOutput(Output output) {} public boolean isPreviewEnabled() { return false; } - - @Override - public @Nullable String getStageName() { - return null; - } } From dec0c3ce1cf525cd81f8c7f2b72c7d6886762c0c Mon Sep 17 00:00:00 2001 From: "igor.krasavin" Date: Wed, 23 Mar 2022 14:47:44 +0300 Subject: [PATCH 20/31] Fix style. --- .../apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java index dcdf7ea9f7f1..ebab7d79fdf6 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java @@ -19,7 +19,6 @@ import io.cdap.cdap.api.data.batch.Output; import io.cdap.cdap.etl.api.batch.BatchSinkContext; -import javax.annotation.Nullable; /** * Class BatchSinkContextWrapper is a class for creating context object of different CDAP classes From 4cacd1c9c284f1aceb01d613944788e4735ebf11 Mon Sep 17 00:00:00 2001 From: "igor.krasavin" Date: Wed, 23 Mar 2022 15:23:59 +0300 Subject: [PATCH 21/31] Determine dependencies. --- sdks/java/io/cdap/build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 30e2ba7d6eb3..3f81f9a24dff 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -51,6 +51,8 @@ dependencies { implementation "com.fasterxml.jackson.core:jackson-databind:2.13.0" implementation "io.cdap.cdap:cdap-etl-api:6.5.1" implementation "io.cdap.cdap:cdap-etl-api-spark:6.2.0" + implementation "io.cdap.cdap:cdap-api-common:6.5.1" + implementation "org.apache.tephra:tephra-api:0.15.0-incubating" testImplementation library.java.vendored_guava_26_0_jre testImplementation "com.github.data-integrations:salesforce:v1.3.9" testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") From 8ba78c9dc971f732948edc9d1c763a1f5091a9ad Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 30 Mar 2022 17:15:54 +0400 Subject: [PATCH 22/31] [BEAM-14048] Use CDAP InstantiatorFactory for creating config objects --- .../beam/gradle/BeamModulePlugin.groovy | 1 + .../src/main/resources/beam/suppressions.xml | 1 + sdks/java/io/cdap/build.gradle | 2 + .../cdap/PluginConfigInstantiationUtils.java | 57 ++----------------- 4 files changed, 10 insertions(+), 51 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index c7c1c858542c..73c5e6e09e69 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -537,6 +537,7 @@ class BeamModulePlugin implements Plugin { cassandra_driver_core : "com.datastax.cassandra:cassandra-driver-core:$cassandra_driver_version", cassandra_driver_mapping : "com.datastax.cassandra:cassandra-driver-mapping:$cassandra_driver_version", cdap_api : "io.cdap.cdap:cdap-api:$cdap_version", + cdap_common : "io.cdap.cdap:cdap-common:$cdap_version", cdap_etl_api : "io.cdap.cdap:cdap-etl-api:$cdap_version", checker_qual : "org.checkerframework:checker-qual:$checkerframework_version", classgraph : "io.github.classgraph:classgraph:$classgraph_version", diff --git a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml index 63e15e0199d4..c050bc5540a0 100644 --- a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml +++ b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml @@ -89,6 +89,7 @@ + diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index c47fbbb1214f..b836d8133a31 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -44,7 +44,9 @@ allprojects { } dependencies { + implementation library.java.guava implementation library.java.cdap_api + implementation library.java.cdap_common implementation library.java.jackson_core implementation library.java.jackson_databind implementation library.java.slf4j_api diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index 187b859217f2..f354b2d15b0d 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.io.cdap; +import com.google.common.reflect.TypeToken; import io.cdap.cdap.api.annotation.Name; import io.cdap.cdap.api.plugin.PluginConfig; -import java.lang.reflect.Constructor; +import io.cdap.cdap.common.lang.InstantiatorFactory; import java.lang.reflect.Field; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Modifier; import java.util.ArrayList; import java.util.Arrays; @@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory; /** Class for getting any filled {@link PluginConfig} configuration object. */ -@SuppressWarnings({"unchecked", "assignment.type.incompatible"}) +@SuppressWarnings({"assignment.type.incompatible", "UnstableApiUsage"}) public class PluginConfigInstantiationUtils { private static final Logger LOG = LoggerFactory.getLogger(PluginConfigInstantiationUtils.class); @@ -63,7 +63,9 @@ public class PluginConfigInstantiationUtils { .collect(Collectors.toList())); currClass = currClass.getSuperclass(); } - T config = getEmptyObjectFromDefaultValues(configClass); + InstantiatorFactory instantiatorFactory = new InstantiatorFactory(false); + + T config = instantiatorFactory.get(TypeToken.of(configClass)).create(); if (config != null) { for (Field field : allFields) { @@ -86,51 +88,4 @@ public class PluginConfigInstantiationUtils { } return config; } - - /** @return empty {@link Object} of {@param tClass} */ - private static @Nullable T getEmptyObjectFromDefaultValues(Class tClass) { - for (Constructor constructor : tClass.getDeclaredConstructors()) { - constructor.setAccessible(true); - 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.warn("Can not instantiate an empty object", e); - } - } - return null; - } - - /** @return default value for given {@param tClass} if it's primitive, otherwise returns null */ - private static @Nullable Object getDefaultValue(@Nullable Class tClass) { - 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; - } } From 970fe7989f16f8f6b4f90a06b100a042265917e7 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 30 Mar 2022 18:26:37 +0400 Subject: [PATCH 23/31] [BEAM-14048] Suppress warning --- .../apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index f354b2d15b0d..d39a321fa4fb 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory; /** Class for getting any filled {@link PluginConfig} configuration object. */ -@SuppressWarnings({"assignment.type.incompatible", "UnstableApiUsage"}) +@SuppressWarnings({"assignment.type.incompatible", "UnstableApiUsage", "return.type.incompatible"}) public class PluginConfigInstantiationUtils { private static final Logger LOG = LoggerFactory.getLogger(PluginConfigInstantiationUtils.class); From 6b7549afb9fc1b9cacdccfec0db350595c991216 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Thu, 7 Apr 2022 14:49:15 +0400 Subject: [PATCH 24/31] [BEAM-14081] Refactoring --- .../beam/gradle/BeamModulePlugin.groovy | 6 ++++ sdks/java/io/cdap/build.gradle | 33 +++++++------------ .../sdk/io/cdap/context/BatchContextImpl.java | 10 +++--- .../io/cdap/context/BatchSinkContextImpl.java | 5 +-- .../cdap/context/BatchSourceContextImpl.java | 5 +-- .../cdap/context/FailureCollectorWrapper.java | 2 +- .../context/StreamingSourceContextImpl.java | 10 +++--- .../io/cdap/context/BatchContextImplTest.java | 4 +-- .../context/FailureCollectorWrapperTest.java | 5 ++- 9 files changed, 34 insertions(+), 46 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 08b3dca52a3d..02d6b25e9d45 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -451,6 +451,7 @@ class BeamModulePlugin implements Plugin { def aws_java_sdk_version = "1.12.135" def aws_java_sdk2_version = "2.17.106" def cassandra_driver_version = "3.10.2" + def cdap_version = "6.5.1" def checkerframework_version = "3.10.0" def classgraph_version = "4.8.104" def errorprone_version = "2.10.0" @@ -535,6 +536,10 @@ class BeamModulePlugin implements Plugin { bigdataoss_util : "com.google.cloud.bigdataoss:util:$google_cloud_bigdataoss_version", cassandra_driver_core : "com.datastax.cassandra:cassandra-driver-core:$cassandra_driver_version", cassandra_driver_mapping : "com.datastax.cassandra:cassandra-driver-mapping:$cassandra_driver_version", + cdap_api : "io.cdap.cdap:cdap-api:$cdap_version", + cdap_api_commons : "io.cdap.cdap:cdap-api-common:$cdap_version", + cdap_etl_api : "io.cdap.cdap:cdap-etl-api:$cdap_version", + cdap_etl_api_spark : "io.cdap.cdap:cdap-etl-api-spark:$cdap_version", checker_qual : "org.checkerframework:checker-qual:$checkerframework_version", classgraph : "io.github.classgraph:classgraph:$classgraph_version", commons_codec : "commons-codec:commons-codec:1.15", @@ -680,6 +685,7 @@ class BeamModulePlugin implements Plugin { spark3_sql : "org.apache.spark:spark-sql_2.12:$spark3_version", spark3_streaming : "org.apache.spark:spark-streaming_2.12:$spark3_version", stax2_api : "org.codehaus.woodstox:stax2-api:4.2.1", + tephra : "org.apache.tephra:tephra-api:0.15.0-incubating", testcontainers_base : "org.testcontainers:testcontainers:$testcontainers_version", testcontainers_clickhouse : "org.testcontainers:clickhouse:$testcontainers_version", testcontainers_elasticsearch : "org.testcontainers:elasticsearch:$testcontainers_version", diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 3f81f9a24dff..2130baca7e0a 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -44,29 +44,18 @@ allprojects { } dependencies { - implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.cdap_api + implementation library.java.cdap_api_commons + implementation library.java.cdap_etl_api + implementation library.java.cdap_etl_api_spark + implementation library.java.jackson_core + implementation library.java.jackson_databind 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" - implementation "io.cdap.cdap:cdap-etl-api:6.5.1" - implementation "io.cdap.cdap:cdap-etl-api-spark:6.2.0" - implementation "io.cdap.cdap:cdap-api-common:6.5.1" - implementation "org.apache.tephra:tephra-api:0.15.0-incubating" - testImplementation library.java.vendored_guava_26_0_jre + implementation library.java.tephra + implementation project(path: ":sdks:java:core", configuration: "shadow") testImplementation "com.github.data-integrations:salesforce:v1.3.9" - 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.cdap_etl_api + testImplementation library.java.vendored_guava_26_0_jre testImplementation library.java.junit - testImplementation library.java.mockito_core - testImplementation library.java.testcontainers_kafka - testImplementation library.java.testcontainers_gcloud + testImplementation project(path: ":runners:direct-java", configuration: "shadow") } diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java index a1d384061d10..f8eb93c81d33 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java @@ -37,14 +37,12 @@ import java.sql.Timestamp; import java.util.List; import java.util.Map; -import javax.annotation.Nullable; /** - * Class OperationContext is a common class for Batch, Sink and Stream CDAP wrapper classes that use - * it to provide common details. + * Common class for Batch, Sink and Stream CDAP wrapper classes that used to provide common details. */ @SuppressWarnings({"TypeParameterUnusedInFormals", "nullness"}) -public class BatchContextImpl implements BatchContext { +public abstract class BatchContextImpl implements BatchContext { private final FailureCollectorWrapper failureCollector = new FailureCollectorWrapper(); @@ -101,12 +99,12 @@ public Schema getInputSchema() { } @Override - public @Nullable Map getInputSchemas() { + public Map getInputSchemas() { return null; } @Override - public @Nullable Schema getOutputSchema() { + public Schema getOutputSchema() { return null; } diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java index ebab7d79fdf6..f0374f7793df 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java @@ -20,10 +20,7 @@ import io.cdap.cdap.api.data.batch.Output; import io.cdap.cdap.etl.api.batch.BatchSinkContext; -/** - * Class BatchSinkContextWrapper is a class for creating context object of different CDAP classes - * with batch sink type. - */ +/** Class for creating context object of different CDAP classes with batch sink type. */ public class BatchSinkContextImpl extends BatchContextImpl implements BatchSinkContext { @Override diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java index 0fed016d2bd6..c8a5fe23801e 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java @@ -20,10 +20,7 @@ import io.cdap.cdap.api.data.batch.Input; import io.cdap.cdap.etl.api.batch.BatchSourceContext; -/** - * Class BatchSourceContextWrapper is a class for creating context object of different CDAP classes - * with batch source type. - */ +/** Class for creating context object of different CDAP classes with batch source type. */ public class BatchSourceContextImpl extends BatchContextImpl implements BatchSourceContext { @Override diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java index d697909d02ef..ce7e98c801b3 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java @@ -23,7 +23,7 @@ import java.util.ArrayList; import javax.annotation.Nullable; -/** Class FailureCollectorWrapper is a class for collecting ValidationFailure. */ +/** Class for collecting {@link ValidationFailure}s. */ public class FailureCollectorWrapper implements FailureCollector { private ArrayList failuresCollection; diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java index b357a9291041..7c09ba19f5fa 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java @@ -23,13 +23,15 @@ import javax.annotation.Nullable; import org.apache.tephra.TransactionFailureException; -/** - * Class StreamingSourceContextWrapper is a class for creating context object of different CDAP - * classes with stream source type. - */ +/** Class for creating context object of different CDAP classes with stream source type. */ public class StreamingSourceContextImpl extends BatchContextImpl implements StreamingSourceContext { @Override public void registerLineage(String referenceName, @Nullable Schema schema) throws DatasetManagementException, TransactionFailureException {} + + @Override + public boolean isPreviewEnabled() { + return false; + } } diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java index 6e58d832e8dd..55d34c654e8c 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java @@ -35,7 +35,7 @@ public class BatchContextImplTest { public void getLogicalStartTime() { /** arrange */ Timestamp expectedStartTime = new Timestamp(System.currentTimeMillis()); - BatchContextImpl context = new BatchContextImpl(); + BatchSinkContextImpl context = new BatchSinkContextImpl(); /** act */ long actualStartTime = context.getLogicalStartTime(); @@ -47,7 +47,7 @@ public void getLogicalStartTime() { @Test public void getFailureCollector() { /** arrange */ - BatchContextImpl context = new BatchContextImpl(); + BatchSourceContextImpl context = new BatchSourceContextImpl(); /** act */ FailureCollector failureCollector = context.getFailureCollector(); diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java index 0e35c8a06a59..828ca49425dd 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java @@ -41,7 +41,7 @@ public void addFailure() { failureCollectorWrapper.addFailure(error.getMessage(), null); /** assert */ - assertThrows(ValidationException.class, () -> failureCollectorWrapper.getOrThrowException()); + assertThrows(ValidationException.class, failureCollectorWrapper::getOrThrowException); } @Test @@ -58,8 +58,7 @@ public void getOrThrowException() { /** act && assert */ ValidationException e = - assertThrows( - ValidationException.class, () -> failureCollectorWrapper.getOrThrowException()); + assertThrows(ValidationException.class, failureCollectorWrapper::getOrThrowException); assertEquals(expectedMessage, e.getMessage()); // A case when return ValidationException with empty collector From d7d2042904d1f85d199a6d2b14699f740fc28ee4 Mon Sep 17 00:00:00 2001 From: Elizaveta Lomteva Date: Mon, 18 Apr 2022 16:15:19 +0400 Subject: [PATCH 25/31] Update maven repo --- sdks/java/io/cdap/build.gradle | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index b836d8133a31..1b9b41c8cf8e 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -39,19 +39,19 @@ interface for integration with CDAP plugins.""" allprojects { repositories { - maven { url 'https://jitpack.io' } + maven { url 'https://oss.sonatype.org/content/repositories/snapshots/' } } } dependencies { - implementation library.java.guava + implementation library.java.guavaz implementation library.java.cdap_api implementation library.java.cdap_common implementation library.java.jackson_core implementation library.java.jackson_databind implementation library.java.slf4j_api implementation project(path: ":sdks:java:core", configuration: "shadow") - testImplementation "com.github.data-integrations:salesforce:v1.3.9" + testImplementation "io.cdap.plugin:salesforce-plugins:1.5.0-SNAPSHOT" testImplementation library.java.cdap_etl_api testImplementation library.java.vendored_guava_26_0_jre testImplementation library.java.junit From 129cffed8e40867c7bd7a3b3a16eab625a67ac87 Mon Sep 17 00:00:00 2001 From: Elizaveta Lomteva <57974525+Lizzfox@users.noreply.github.com> Date: Mon, 18 Apr 2022 18:21:39 +0400 Subject: [PATCH 26/31] Update build.gradle --- sdks/java/io/cdap/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 1b9b41c8cf8e..f24305ab89b9 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -44,7 +44,7 @@ allprojects { } dependencies { - implementation library.java.guavaz + implementation library.java.guava implementation library.java.cdap_api implementation library.java.cdap_common implementation library.java.jackson_core From 855122ac7ee3bc3da1e86b731ccebb0ae4d3f3d3 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Thu, 28 Apr 2022 17:33:15 +0400 Subject: [PATCH 27/31] [BEAM-14081] Refactoring --- .../sdk/io/cdap/context/BatchContextImpl.java | 25 ++++++++++++++++--- .../io/cdap/context/BatchSinkContextImpl.java | 5 +++- .../cdap/context/BatchSourceContextImpl.java | 9 +++++-- .../cdap/context/FailureCollectorWrapper.java | 2 +- .../context/StreamingSourceContextImpl.java | 5 +++- .../io/cdap/context/BatchContextImplTest.java | 4 +-- .../context/FailureCollectorWrapperTest.java | 5 ++-- 7 files changed, 42 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java index f8eb93c81d33..006f7f46c22c 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.cdap.context; import io.cdap.cdap.api.data.DatasetInstantiationException; +import io.cdap.cdap.api.data.batch.InputFormatProvider; import io.cdap.cdap.api.data.schema.Schema; import io.cdap.cdap.api.dataset.Dataset; import io.cdap.cdap.api.dataset.DatasetManagementException; @@ -30,6 +31,7 @@ import io.cdap.cdap.etl.api.FailureCollector; import io.cdap.cdap.etl.api.Lookup; import io.cdap.cdap.etl.api.StageMetrics; +import io.cdap.cdap.etl.api.SubmitterLifecycle; import io.cdap.cdap.etl.api.action.SettableArguments; import io.cdap.cdap.etl.api.batch.BatchContext; import io.cdap.cdap.etl.api.lineage.field.FieldOperation; @@ -37,17 +39,29 @@ import java.sql.Timestamp; import java.util.List; import java.util.Map; +import javax.annotation.Nullable; /** - * Common class for Batch, Sink and Stream CDAP wrapper classes that used to provide common details. + * Class OperationContext is a common class for Batch, Sink and Stream CDAP wrapper classes that use + * it to provide common details. */ @SuppressWarnings({"TypeParameterUnusedInFormals", "nullness"}) -public abstract class BatchContextImpl implements BatchContext { +public class BatchContextImpl implements BatchContext { private final FailureCollectorWrapper failureCollector = new FailureCollectorWrapper(); + /** + * This should be set after {@link SubmitterLifecycle#prepareRun(Object)} call with passing this + * context object as a param. + */ + protected InputFormatProvider inputFormatProvider; + private final Timestamp startTime = new Timestamp(System.currentTimeMillis()); + public InputFormatProvider getInputFormatProvider() { + return inputFormatProvider; + } + @Override public String getStageName() { return null; @@ -93,18 +107,19 @@ public T newPluginInstance(String pluginId) throws InstantiationException { return null; } + @Nullable @Override public Schema getInputSchema() { return null; } @Override - public Map getInputSchemas() { + public @Nullable Map getInputSchemas() { return null; } @Override - public Schema getOutputSchema() { + public @Nullable Schema getOutputSchema() { return null; } @@ -132,11 +147,13 @@ public FailureCollector getFailureCollector() { return this.failureCollector; } + @Nullable @Override public URL getServiceURL(String applicationId, String serviceId) { return null; } + @Nullable @Override public URL getServiceURL(String serviceId) { return null; diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java index f0374f7793df..ebab7d79fdf6 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java @@ -20,7 +20,10 @@ import io.cdap.cdap.api.data.batch.Output; import io.cdap.cdap.etl.api.batch.BatchSinkContext; -/** Class for creating context object of different CDAP classes with batch sink type. */ +/** + * Class BatchSinkContextWrapper is a class for creating context object of different CDAP classes + * with batch sink type. + */ public class BatchSinkContextImpl extends BatchContextImpl implements BatchSinkContext { @Override diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java index c8a5fe23801e..6a3263b7fdae 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java @@ -20,11 +20,16 @@ import io.cdap.cdap.api.data.batch.Input; import io.cdap.cdap.etl.api.batch.BatchSourceContext; -/** Class for creating context object of different CDAP classes with batch source type. */ +/** + * Class BatchSourceContextWrapper is a class for creating context object of different CDAP classes + * with batch source type. + */ public class BatchSourceContextImpl extends BatchContextImpl implements BatchSourceContext { @Override - public void setInput(Input input) {} + public void setInput(Input input) { + this.inputFormatProvider = ((Input.InputFormatProviderInput) input).getInputFormatProvider(); + } @Override public boolean isPreviewEnabled() { diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java index ce7e98c801b3..d697909d02ef 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java @@ -23,7 +23,7 @@ import java.util.ArrayList; import javax.annotation.Nullable; -/** Class for collecting {@link ValidationFailure}s. */ +/** Class FailureCollectorWrapper is a class for collecting ValidationFailure. */ public class FailureCollectorWrapper implements FailureCollector { private ArrayList failuresCollection; diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java index 7c09ba19f5fa..cb4515785e97 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java @@ -23,7 +23,10 @@ import javax.annotation.Nullable; import org.apache.tephra.TransactionFailureException; -/** Class for creating context object of different CDAP classes with stream source type. */ +/** + * Class StreamingSourceContextWrapper is a class for creating context object of different CDAP + * classes with stream source type. + */ public class StreamingSourceContextImpl extends BatchContextImpl implements StreamingSourceContext { @Override diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java index 55d34c654e8c..6e58d832e8dd 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java @@ -35,7 +35,7 @@ public class BatchContextImplTest { public void getLogicalStartTime() { /** arrange */ Timestamp expectedStartTime = new Timestamp(System.currentTimeMillis()); - BatchSinkContextImpl context = new BatchSinkContextImpl(); + BatchContextImpl context = new BatchContextImpl(); /** act */ long actualStartTime = context.getLogicalStartTime(); @@ -47,7 +47,7 @@ public void getLogicalStartTime() { @Test public void getFailureCollector() { /** arrange */ - BatchSourceContextImpl context = new BatchSourceContextImpl(); + BatchContextImpl context = new BatchContextImpl(); /** act */ FailureCollector failureCollector = context.getFailureCollector(); diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java index 828ca49425dd..0e35c8a06a59 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java @@ -41,7 +41,7 @@ public void addFailure() { failureCollectorWrapper.addFailure(error.getMessage(), null); /** assert */ - assertThrows(ValidationException.class, failureCollectorWrapper::getOrThrowException); + assertThrows(ValidationException.class, () -> failureCollectorWrapper.getOrThrowException()); } @Test @@ -58,7 +58,8 @@ public void getOrThrowException() { /** act && assert */ ValidationException e = - assertThrows(ValidationException.class, failureCollectorWrapper::getOrThrowException); + assertThrows( + ValidationException.class, () -> failureCollectorWrapper.getOrThrowException()); assertEquals(expectedMessage, e.getMessage()); // A case when return ValidationException with empty collector From f440da9cf0a65c6c8d2cabad2495bb98b04e9cfc Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 4 May 2022 14:32:22 +0400 Subject: [PATCH 28/31] [BEAM-14048] Use ServiceNow CDAP dependency from Maven central --- .../beam/gradle/BeamModulePlugin.groovy | 1 + sdks/java/io/cdap/build.gradle | 8 +- .../beam/sdk/io/cdap/ConfigWrapperTest.java | 88 ++++++++++--------- .../PluginConfigInstantiationUtilsTest.java | 54 ++++++------ .../resources/salesforce_test_params.json | 10 --- .../resources/service_now_test_params.json | 11 +++ 6 files changed, 87 insertions(+), 85 deletions(-) delete mode 100644 sdks/java/io/cdap/src/test/resources/salesforce_test_params.json create mode 100644 sdks/java/io/cdap/src/test/resources/service_now_test_params.json diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 73c5e6e09e69..acc0f5cce7ae 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -539,6 +539,7 @@ class BeamModulePlugin implements Plugin { cdap_api : "io.cdap.cdap:cdap-api:$cdap_version", cdap_common : "io.cdap.cdap:cdap-common:$cdap_version", cdap_etl_api : "io.cdap.cdap:cdap-etl-api:$cdap_version", + cdap_plugin_service_now : "io.cdap.plugin:servicenow-plugins:1.1.0", checker_qual : "org.checkerframework:checker-qual:$checkerframework_version", classgraph : "io.github.classgraph:classgraph:$classgraph_version", commons_codec : "commons-codec:commons-codec:1.15", diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index b836d8133a31..4ef361924d75 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -37,12 +37,6 @@ interface for integration with CDAP plugins.""" * for details. */ -allprojects { - repositories { - maven { url 'https://jitpack.io' } - } -} - dependencies { implementation library.java.guava implementation library.java.cdap_api @@ -51,7 +45,7 @@ dependencies { implementation library.java.jackson_databind implementation library.java.slf4j_api implementation project(path: ":sdks:java:core", configuration: "shadow") - testImplementation "com.github.data-integrations:salesforce:v1.3.9" + testImplementation library.java.cdap_plugin_service_now testImplementation library.java.cdap_etl_api testImplementation library.java.vendored_guava_26_0_jre testImplementation library.java.junit diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java index e5b33de83113..55f3ab49050f 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/ConfigWrapperTest.java @@ -21,9 +21,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; -import io.cdap.plugin.salesforce.SalesforceConstants; -import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceSourceConfig; -import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; +import io.cdap.plugin.servicenow.source.ServiceNowSourceConfig; +import io.cdap.plugin.servicenow.source.util.ServiceNowConstants; import java.io.File; import java.util.Map; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; @@ -39,43 +38,46 @@ public class ConfigWrapperTest { private static final Logger LOG = LoggerFactory.getLogger(ConfigWrapperTest.class); - private static final ImmutableMap TEST_SALESFORCE_PARAMS_MAP = + private static final ImmutableMap TEST_SERVICE_NOW_PARAMS_MAP = ImmutableMap.builder() - .put("sObjectName", "sObject") - .put("datetimeAfter", "datetime") - .put("consumerKey", "key") - .put("consumerSecret", "secret") - .put("username", "user") - .put("password", "password") - .put("loginUrl", "https://www.google.com") + .put(ServiceNowConstants.PROPERTY_CLIENT_ID, "clientId") + .put(ServiceNowConstants.PROPERTY_CLIENT_SECRET, "clientSecret") + .put(ServiceNowConstants.PROPERTY_API_ENDPOINT, "https://www.google.com") + .put(ServiceNowConstants.PROPERTY_QUERY_MODE, "Table") + .put(ServiceNowConstants.PROPERTY_USER, "user") + .put(ServiceNowConstants.PROPERTY_PASSWORD, "password") + .put(ServiceNowConstants.PROPERTY_TABLE_NAME, "tableName") + .put(ServiceNowConstants.PROPERTY_VALUE_TYPE, "Actual") .put("referenceName", "oldReference") .build(); - private static final String TEST_SALESFORCE_PARAMS_JSON_STRING = + + private static final String TEST_SERVICE_NOW_PARAMS_JSON_STRING = "{\n" - + "\"sObjectName\": \"sObject\",\n" - + "\"datetimeAfter\": \"datetime\",\n" - + "\"consumerKey\": \"key\",\n" - + "\"consumerSecret\": \"secret\",\n" - + "\"username\": \"user\",\n" + + "\"clientId\": \"clientId\",\n" + + "\"clientSecret\": \"clientSecret\",\n" + + "\"restApiEndpoint\": \"https://www.google.com\",\n" + + "\"queryMode\": \"Table\",\n" + + "\"user\": \"user\",\n" + "\"password\": \"password\",\n" - + "\"loginUrl\": \"https://www.google.com\",\n" - + "\"referenceName\": \"reference\"\n" + + "\"tableName\": \"tableName\",\n" + + "\"valueType\": \"Actual\",\n" + + "\"referenceName\": \"oldReference\"\n" + "}"; - private static final String SALESFORCE_TEST_PARAMS_JSON = - "src/test/resources/salesforce_test_params.json"; + private static final String SERVICE_NOW_TEST_PARAMS_JSON = + "src/test/resources/service_now_test_params.json"; public static final String REFERENCE_NAME_PARAM_NAME = "referenceName"; @Test public void testBuildingPluginConfigFromParamsMap() { try { String newReferenceName = "new reference name"; - SalesforceSourceConfig config = - new ConfigWrapper<>(SalesforceSourceConfig.class) - .withParams(TEST_SALESFORCE_PARAMS_MAP) + ServiceNowSourceConfig config = + new ConfigWrapper<>(ServiceNowSourceConfig.class) + .withParams(TEST_SERVICE_NOW_PARAMS_MAP) .setParam("referenceName", newReferenceName) .build(); assertNotNull(config); - validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, config); + validateServiceNowConfigObject(TEST_SERVICE_NOW_PARAMS_MAP, config); assertEquals(newReferenceName, config.referenceName); } catch (Exception e) { LOG.error("Error occurred while building the config object", e); @@ -87,13 +89,13 @@ public void testBuildingPluginConfigFromParamsMap() { public void testBuildingPluginConfigFromJsonFile() { try { String newReferenceName = "new reference name"; - SalesforceSourceConfig config = - new ConfigWrapper<>(SalesforceSourceConfig.class) - .fromJsonFile(new File(SALESFORCE_TEST_PARAMS_JSON)) + ServiceNowSourceConfig config = + new ConfigWrapper<>(ServiceNowSourceConfig.class) + .fromJsonFile(new File(SERVICE_NOW_TEST_PARAMS_JSON)) .setParam(REFERENCE_NAME_PARAM_NAME, newReferenceName) .build(); assertNotNull(config); - validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, config); + validateServiceNowConfigObject(TEST_SERVICE_NOW_PARAMS_MAP, config); assertEquals(newReferenceName, config.referenceName); } catch (Exception e) { LOG.error("Error occurred while building the config object", e); @@ -105,13 +107,13 @@ public void testBuildingPluginConfigFromJsonFile() { public void testBuildingPluginConfigFromJsonString() { try { String newReferenceName = "new reference name"; - SalesforceSourceConfig config = - new ConfigWrapper<>(SalesforceSourceConfig.class) - .fromJsonString(TEST_SALESFORCE_PARAMS_JSON_STRING) + ServiceNowSourceConfig config = + new ConfigWrapper<>(ServiceNowSourceConfig.class) + .fromJsonString(TEST_SERVICE_NOW_PARAMS_JSON_STRING) .setParam(REFERENCE_NAME_PARAM_NAME, newReferenceName) .build(); assertNotNull(config); - validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, config); + validateServiceNowConfigObject(TEST_SERVICE_NOW_PARAMS_MAP, config); assertEquals(newReferenceName, config.referenceName); } catch (Exception e) { LOG.error("Error occurred while building the config object", e); @@ -119,17 +121,19 @@ public void testBuildingPluginConfigFromJsonString() { } } - private static void validateSalesforceConfigObject( - Map params, SalesforceSourceConfig config) { + private static void validateServiceNowConfigObject( + Map params, ServiceNowSourceConfig config) { + assertEquals(params.get(ServiceNowConstants.PROPERTY_CLIENT_ID), config.getClientId()); + assertEquals(params.get(ServiceNowConstants.PROPERTY_CLIENT_SECRET), config.getClientSecret()); assertEquals( - params.get(SalesforceSourceConstants.PROPERTY_DATETIME_AFTER), config.getDatetimeAfter()); + params.get(ServiceNowConstants.PROPERTY_API_ENDPOINT), config.getRestApiEndpoint()); assertEquals( - params.get(SalesforceSourceConstants.PROPERTY_SOBJECT_NAME), config.getSObjectName()); - assertEquals(params.get(SalesforceConstants.PROPERTY_CONSUMER_KEY), config.getConsumerKey()); + params.get(ServiceNowConstants.PROPERTY_QUERY_MODE), config.getQueryMode().getValue()); + assertEquals(params.get(ServiceNowConstants.PROPERTY_USER), config.getUser()); + assertEquals(params.get(ServiceNowConstants.PROPERTY_PASSWORD), config.getPassword()); + assertNotNull(config.getValueType()); assertEquals( - params.get(SalesforceConstants.PROPERTY_CONSUMER_SECRET), config.getConsumerSecret()); - assertEquals(params.get(SalesforceConstants.PROPERTY_USERNAME), config.getUsername()); - assertEquals(params.get(SalesforceConstants.PROPERTY_PASSWORD), config.getPassword()); - assertEquals(params.get(SalesforceConstants.PROPERTY_LOGIN_URL), config.getLoginUrl()); + params.get(ServiceNowConstants.PROPERTY_VALUE_TYPE), config.getValueType().getValueType()); + assertEquals(params.get(ServiceNowConstants.PROPERTY_TABLE_NAME), config.getTableName()); } } diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtilsTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtilsTest.java index 394cf45f5600..90f4e4988064 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtilsTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtilsTest.java @@ -21,9 +21,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; -import io.cdap.plugin.salesforce.SalesforceConstants; -import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceSourceConfig; -import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; +import io.cdap.plugin.servicenow.source.ServiceNowSourceConfig; +import io.cdap.plugin.servicenow.source.util.ServiceNowConstants; import java.util.HashMap; import java.util.Map; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; @@ -40,26 +39,27 @@ public class PluginConfigInstantiationUtilsTest { private static final Logger LOG = LoggerFactory.getLogger(PluginConfigInstantiationUtilsTest.class); - private static final ImmutableMap TEST_SALESFORCE_PARAMS_MAP = + private static final ImmutableMap TEST_SERVICE_NOW_PARAMS_MAP = ImmutableMap.builder() - .put("sObjectName", "sObject") - .put("datetimeAfter", "datetime") - .put("consumerKey", "key") - .put("consumerSecret", "secret") - .put("username", "user") - .put("password", "password") - .put("loginUrl", "https://www.google.com") + .put(ServiceNowConstants.PROPERTY_CLIENT_ID, "clientId") + .put(ServiceNowConstants.PROPERTY_CLIENT_SECRET, "clientSecret") + .put(ServiceNowConstants.PROPERTY_API_ENDPOINT, "https://www.google.com") + .put(ServiceNowConstants.PROPERTY_QUERY_MODE, "Table") + .put(ServiceNowConstants.PROPERTY_USER, "user") + .put(ServiceNowConstants.PROPERTY_PASSWORD, "password") + .put(ServiceNowConstants.PROPERTY_TABLE_NAME, "tableName") + .put(ServiceNowConstants.PROPERTY_VALUE_TYPE, "Actual") .put("referenceName", "oldReference") .build(); @Test public void testBuildingPluginConfigFromParamsMap() { try { - SalesforceSourceConfig config = + ServiceNowSourceConfig config = PluginConfigInstantiationUtils.getPluginConfig( - TEST_SALESFORCE_PARAMS_MAP, SalesforceSourceConfig.class); + TEST_SERVICE_NOW_PARAMS_MAP, ServiceNowSourceConfig.class); assertNotNull(config); - validateSalesforceConfigObject(TEST_SALESFORCE_PARAMS_MAP, config); + validateServiceNowConfigObject(TEST_SERVICE_NOW_PARAMS_MAP, config); } catch (Exception e) { LOG.error("Error occurred while building the config object", e); fail(); @@ -69,9 +69,9 @@ public void testBuildingPluginConfigFromParamsMap() { @Test public void testBuildingPluginConfigFromEmptyParamsMap() { try { - SalesforceSourceConfig config = + ServiceNowSourceConfig config = PluginConfigInstantiationUtils.getPluginConfig( - new HashMap<>(), SalesforceSourceConfig.class); + new HashMap<>(), ServiceNowSourceConfig.class); assertNotNull(config); } catch (Exception e) { LOG.error("Error occurred while building the config object", e); @@ -82,24 +82,26 @@ public void testBuildingPluginConfigFromEmptyParamsMap() { @Test public void testBuildingPluginConfigFromNullClassFail() { try { - PluginConfigInstantiationUtils.getPluginConfig(TEST_SALESFORCE_PARAMS_MAP, null); + PluginConfigInstantiationUtils.getPluginConfig(TEST_SERVICE_NOW_PARAMS_MAP, null); fail(); } catch (IllegalArgumentException e) { assertEquals("Config class must be not null!", e.getMessage()); } } - private static void validateSalesforceConfigObject( - Map params, SalesforceSourceConfig config) { + private static void validateServiceNowConfigObject( + Map params, ServiceNowSourceConfig config) { + assertEquals(params.get(ServiceNowConstants.PROPERTY_CLIENT_ID), config.getClientId()); + assertEquals(params.get(ServiceNowConstants.PROPERTY_CLIENT_SECRET), config.getClientSecret()); assertEquals( - params.get(SalesforceSourceConstants.PROPERTY_DATETIME_AFTER), config.getDatetimeAfter()); + params.get(ServiceNowConstants.PROPERTY_API_ENDPOINT), config.getRestApiEndpoint()); assertEquals( - params.get(SalesforceSourceConstants.PROPERTY_SOBJECT_NAME), config.getSObjectName()); - assertEquals(params.get(SalesforceConstants.PROPERTY_CONSUMER_KEY), config.getConsumerKey()); + params.get(ServiceNowConstants.PROPERTY_QUERY_MODE), config.getQueryMode().getValue()); + assertEquals(params.get(ServiceNowConstants.PROPERTY_USER), config.getUser()); + assertEquals(params.get(ServiceNowConstants.PROPERTY_PASSWORD), config.getPassword()); + assertNotNull(config.getValueType()); assertEquals( - params.get(SalesforceConstants.PROPERTY_CONSUMER_SECRET), config.getConsumerSecret()); - assertEquals(params.get(SalesforceConstants.PROPERTY_USERNAME), config.getUsername()); - assertEquals(params.get(SalesforceConstants.PROPERTY_PASSWORD), config.getPassword()); - assertEquals(params.get(SalesforceConstants.PROPERTY_LOGIN_URL), config.getLoginUrl()); + params.get(ServiceNowConstants.PROPERTY_VALUE_TYPE), config.getValueType().getValueType()); + assertEquals(params.get(ServiceNowConstants.PROPERTY_TABLE_NAME), config.getTableName()); } } diff --git a/sdks/java/io/cdap/src/test/resources/salesforce_test_params.json b/sdks/java/io/cdap/src/test/resources/salesforce_test_params.json deleted file mode 100644 index bd9891e35078..000000000000 --- a/sdks/java/io/cdap/src/test/resources/salesforce_test_params.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "sObjectName": "sObject", - "datetimeAfter": "datetime", - "consumerKey": "key", - "consumerSecret": "secret", - "username": "user", - "password": "password", - "loginUrl": "https://www.google.com", - "referenceName": "reference" -} diff --git a/sdks/java/io/cdap/src/test/resources/service_now_test_params.json b/sdks/java/io/cdap/src/test/resources/service_now_test_params.json new file mode 100644 index 000000000000..96bbdc7383fb --- /dev/null +++ b/sdks/java/io/cdap/src/test/resources/service_now_test_params.json @@ -0,0 +1,11 @@ +{ + "clientId": "clientId", + "clientSecret": "clientSecret", + "restApiEndpoint": "https://www.google.com", + "queryMode": "Table", + "user": "user", + "password": "password", + "tableName": "tableName", + "valueType": "Actual", + "referenceName": "oldReference" +} \ No newline at end of file From 9df1c7552cda5d6890d81007ce5e37f6216eb8c6 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 4 May 2022 17:56:31 +0400 Subject: [PATCH 29/31] [BEAM-14048] Set macroFields --- .../sdk/io/cdap/PluginConfigInstantiationUtils.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java index d39a321fa4fb..f13a3e7f7e92 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java @@ -25,6 +25,7 @@ import java.lang.reflect.Modifier; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -37,6 +38,7 @@ public class PluginConfigInstantiationUtils { private static final Logger LOG = LoggerFactory.getLogger(PluginConfigInstantiationUtils.class); + private static final String MACRO_FIELDS_FIELD_NAME = "macroFields"; /** * Method for instantiating {@link PluginConfig} object of specific class {@param configClass}. @@ -58,8 +60,7 @@ public class PluginConfigInstantiationUtils { while (currClass != null && !currClass.equals(Object.class)) { allFields.addAll( Arrays.stream(currClass.getDeclaredFields()) - .filter( - f -> !Modifier.isStatic(f.getModifiers()) && f.isAnnotationPresent(Name.class)) + .filter(f -> !Modifier.isStatic(f.getModifiers())) .collect(Collectors.toList())); currClass = currClass.getSuperclass(); } @@ -83,6 +84,12 @@ public class PluginConfigInstantiationUtils { } catch (IllegalAccessException e) { LOG.error("Can not set a field with value {}", fieldValue); } + } else if (field.getName().equals(MACRO_FIELDS_FIELD_NAME)) { + try { + field.set(config, Collections.emptySet()); + } catch (IllegalAccessException e) { + LOG.error("Can not set macro fields"); + } } } } From 9e31a72fb80a786660fbf42a5e07fed86485660c Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Thu, 5 May 2022 10:31:43 +0400 Subject: [PATCH 30/31] [BEAM-14081] Fix javadoc --- .../apache/beam/sdk/io/cdap/context/BatchContextImpl.java | 5 +---- .../beam/sdk/io/cdap/context/BatchSinkContextImpl.java | 5 +---- .../beam/sdk/io/cdap/context/BatchSourceContextImpl.java | 5 +---- .../beam/sdk/io/cdap/context/StreamingSourceContextImpl.java | 5 +---- 4 files changed, 4 insertions(+), 16 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java index 006f7f46c22c..9b255e270290 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java @@ -41,10 +41,7 @@ import java.util.Map; import javax.annotation.Nullable; -/** - * Class OperationContext is a common class for Batch, Sink and Stream CDAP wrapper classes that use - * it to provide common details. - */ +/** Class for Batch, Sink and Stream CDAP wrapper classes that use it to provide common details. */ @SuppressWarnings({"TypeParameterUnusedInFormals", "nullness"}) public class BatchContextImpl implements BatchContext { diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java index ebab7d79fdf6..f0374f7793df 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java @@ -20,10 +20,7 @@ import io.cdap.cdap.api.data.batch.Output; import io.cdap.cdap.etl.api.batch.BatchSinkContext; -/** - * Class BatchSinkContextWrapper is a class for creating context object of different CDAP classes - * with batch sink type. - */ +/** Class for creating context object of different CDAP classes with batch sink type. */ public class BatchSinkContextImpl extends BatchContextImpl implements BatchSinkContext { @Override diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java index 6a3263b7fdae..98532936035d 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSourceContextImpl.java @@ -20,10 +20,7 @@ import io.cdap.cdap.api.data.batch.Input; import io.cdap.cdap.etl.api.batch.BatchSourceContext; -/** - * Class BatchSourceContextWrapper is a class for creating context object of different CDAP classes - * with batch source type. - */ +/** Class for creating context object of different CDAP classes with batch source type. */ public class BatchSourceContextImpl extends BatchContextImpl implements BatchSourceContext { @Override diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java index cb4515785e97..7c09ba19f5fa 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/StreamingSourceContextImpl.java @@ -23,10 +23,7 @@ import javax.annotation.Nullable; import org.apache.tephra.TransactionFailureException; -/** - * Class StreamingSourceContextWrapper is a class for creating context object of different CDAP - * classes with stream source type. - */ +/** Class for creating context object of different CDAP classes with stream source type. */ public class StreamingSourceContextImpl extends BatchContextImpl implements StreamingSourceContext { @Override From 2f77cc4980961f3c2094ca17d02fb1943fd86a0d Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 11 May 2022 13:19:37 +0400 Subject: [PATCH 31/31] [BEAM-14081] Make BatchContextImpl class abstract --- .../org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java | 2 +- .../apache/beam/sdk/io/cdap/context/BatchContextImplTest.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java index 9b255e270290..06b174062df0 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java @@ -43,7 +43,7 @@ /** Class for Batch, Sink and Stream CDAP wrapper classes that use it to provide common details. */ @SuppressWarnings({"TypeParameterUnusedInFormals", "nullness"}) -public class BatchContextImpl implements BatchContext { +public abstract class BatchContextImpl implements BatchContext { private final FailureCollectorWrapper failureCollector = new FailureCollectorWrapper(); diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java index 6e58d832e8dd..8f679fe3fc08 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/BatchContextImplTest.java @@ -35,7 +35,7 @@ public class BatchContextImplTest { public void getLogicalStartTime() { /** arrange */ Timestamp expectedStartTime = new Timestamp(System.currentTimeMillis()); - BatchContextImpl context = new BatchContextImpl(); + BatchContextImpl context = new BatchSourceContextImpl(); /** act */ long actualStartTime = context.getLogicalStartTime(); @@ -47,7 +47,7 @@ public void getLogicalStartTime() { @Test public void getFailureCollector() { /** arrange */ - BatchContextImpl context = new BatchContextImpl(); + BatchContextImpl context = new BatchSinkContextImpl(); /** act */ FailureCollector failureCollector = context.getFailureCollector();