[
https://issues.apache.org/jira/browse/BEAM-14048?focusedWorklogId=756333&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-756333
]
ASF GitHub Bot logged work on BEAM-14048:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 13/Apr/22 11:47
Start Date: 13/Apr/22 11:47
Worklog Time Spent: 10m
Work Description: Amar3tto commented on code in PR #17051:
URL: https://github.com/apache/beam/pull/17051#discussion_r849389726
##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/PluginConfigInstantiationUtils.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.cdap;
+
+import io.cdap.cdap.api.annotation.Name;
+import io.cdap.cdap.api.plugin.PluginConfig;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class for getting any filled {@link io.cdap.cdap.api.plugin.PluginConfig}
configuration object.
+ */
+@SuppressWarnings({"unchecked", "assignment.type.incompatible"})
+public class PluginConfigInstantiationUtils {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(PluginConfigInstantiationUtils.class);
+
+ /**
+ * @param params map of config fields, where key is the name of the field,
value must be String or
+ * boxed primitive
+ * @return Config object for given map of arguments and configuration class
+ */
+ public static @Nullable <T extends PluginConfig> T getPluginConfig(
+ Map<String, Object> params, Class<T> configClass) {
+ // Validate configClass
+ if (configClass == null || configClass.isPrimitive() ||
configClass.isArray()) {
+ throw new IllegalArgumentException("Config class must be correct!");
+ }
+ List<Field> allFields = new ArrayList<>();
+ Class<?> currClass = configClass;
+ while (currClass != null && !currClass.equals(Object.class)) {
+ allFields.addAll(
+ Arrays.stream(currClass.getDeclaredFields())
+ .filter(
+ f -> !Modifier.isStatic(f.getModifiers()) &&
f.isAnnotationPresent(Name.class))
+ .collect(Collectors.toList()));
+ currClass = currClass.getSuperclass();
+ }
+ T config = getEmptyObjectOf(configClass);
+
+ if (config != null) {
+ for (Field field : allFields) {
+ field.setAccessible(true);
+
+ Class<?> fieldType = field.getType();
+
+ Name declaredAnnotation = field.getDeclaredAnnotation(Name.class);
+ Object fieldValue =
+ declaredAnnotation != null ?
params.get(declaredAnnotation.value()) : null;
+
+ if (fieldValue != null && fieldType.equals(fieldValue.getClass())) {
+ try {
+ field.set(config, fieldValue);
+ } catch (IllegalAccessException e) {
+ LOG.error("Can not set a field", e);
+ }
+ }
+ }
+ }
+ return config;
+ }
+
+ /** @return empty {@link Object} of {@param tClass} */
+ private static @Nullable <T> T getEmptyObjectOf(Class<T> tClass) {
+ for (Constructor<?> constructor : tClass.getDeclaredConstructors()) {
+ constructor.setAccessible(true);
Review Comment:
Done. Used CDAP InstantiatorFactory for creating config objects.
Issue Time Tracking
-------------------
Worklog Id: (was: 756333)
Time Spent: 30h 50m (was: 30h 40m)
> [CdapIO] Design and implement ConfigWrapper for building CDAP PluginConfigs
> ---------------------------------------------------------------------------
>
> Key: BEAM-14048
> URL: https://issues.apache.org/jira/browse/BEAM-14048
> Project: Beam
> Issue Type: Task
> Components: io-java-cdap
> Reporter: Elizaveta Lomteva
> Assignee: Vitaly Terentyev
> Priority: P2
> Labels: cdap-io-sprint-1
> Time Spent: 30h 50m
> Remaining Estimate: 0h
>
> h3. Context:
> CDAP plugins use their configuration classes (ex.
> {{{}SalesforceSinkConfig{}}}) to set the required parameters to prepare and
> run a pipeline in Source or Sink classes (ex. {{{}SalesforceBatchSink{}}}).
> The config instance is accepted as a Source/Sink constructor argument.
> The configuration classes are usually placed in the {{{}cdap/plugin/\{<plugin
> name>}/plugin/\{sink|source}/{batch|streaming{}}}} folder (ex. [Salesforce
> repo|https://github.com/data-integrations/salesforce/tree/d0d83fd82bbdbbb7ef07168f3b1698eebdc90c3e/src/main/java/io/cdap/plugin/salesforce/plugin]).
> h3. Task Description:
> Required to design wrapper class(es) for the CDAP plugin config class so that
> the CDAP plugin config instance can be created in the Apache Beam connector
> based on user input and used when creating CDAP Source or Sink.
> h3. Acceptance criteria:
> Design and source code of CDAP Config wrapper that will
> # accept custom parameters as constructor arguments (JSON/string or
> something else)
> # return an instance of plugins configuration class (ex.
> {{{}SalesforceSinkConfig{}}})
--
This message was sent by Atlassian Jira
(v8.20.1#820001)