You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by hu...@apache.org on 2020/06/04 00:47:14 UTC

[incubator-gobblin] branch master updated: [GOBBLIN-1179] Add typed config in salesforce

This is an automated email from the ASF dual-hosted git repository.

hutran pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-gobblin.git


The following commit(s) were added to refs/heads/master by this push:
     new 5b10709  [GOBBLIN-1179] Add typed config in salesforce
5b10709 is described below

commit 5b1070992d906824889eda4c7041bb6d28ae549b
Author: Alex Li <al...@linkedin.com>
AuthorDate: Wed Jun 3 17:47:07 2020 -0700

    [GOBBLIN-1179] Add typed config in salesforce
    
    Closes #2910 from arekusuri/typedconfig
---
 .gitignore                                         |   2 +
 .../gobblin/salesforce/QueryBasedSourceConfig.java |  29 +++++
 .../salesforce/SalesforceConfigurationKeys.java    |   6 -
 .../gobblin/salesforce/SalesforceExtractor.java    |  12 +-
 .../org/apache/gobblin/salesforce/SfConfig.java    |  39 ++++++
 .../java/org/apache/gobblin/typedconfig/Alias.java |  30 +++++
 .../apache/gobblin/typedconfig/ConstraintUtil.java | 135 +++++++++++++++++++++
 .../org/apache/gobblin/typedconfig/Default.java    |  30 +++++
 .../java/org/apache/gobblin/typedconfig/Key.java   |  31 +++++
 .../apache/gobblin/typedconfig/TypedConfig.java    | 127 +++++++++++++++++++
 .../typedconfig/compiletime/EnumOptions.java       |  30 +++++
 .../gobblin/typedconfig/compiletime/IntRange.java  |  30 +++++
 .../gobblin/typedconfig/compiletime/LongRange.java |  30 +++++
 .../typedconfig/compiletime/StringRegex.java       |  30 +++++
 14 files changed, 548 insertions(+), 13 deletions(-)

diff --git a/.gitignore b/.gitignore
index 4135edf..a6075f2 100644
--- a/.gitignore
+++ b/.gitignore
@@ -80,7 +80,9 @@ FsDatasetStateStoreTest/
 GobblinHelixTaskTest/
 commit-sequence-store-test/
 gobblin-test-harness/src/test/resources/runtime_test/state_store/
+
 gobblin-integration-test-work-dir/
 
 gobblin-test-utils/src/main/gen-avro/
 gobblin-test-utils/src/main/gen-proto/
+
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/QueryBasedSourceConfig.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/QueryBasedSourceConfig.java
new file mode 100644
index 0000000..20c24fb
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/QueryBasedSourceConfig.java
@@ -0,0 +1,29 @@
+/*
+ * 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.gobblin.salesforce;
+
+import java.util.Properties;
+import org.apache.gobblin.typedconfig.TypedConfig;
+
+
+public class QueryBasedSourceConfig extends TypedConfig {
+
+  public QueryBasedSourceConfig(Properties prop) {
+    super(prop);
+  }
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceConfigurationKeys.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceConfigurationKeys.java
index b40e0e8..a01285a 100644
--- a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceConfigurationKeys.java
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceConfigurationKeys.java
@@ -25,8 +25,6 @@ public final class SalesforceConfigurationKeys {
   }
   public static final String SOURCE_QUERYBASED_SALESFORCE_IS_SOFT_DELETES_PULL_DISABLED =
       "source.querybased.salesforce.is.soft.deletes.pull.disabled";
-  public static final int DEFAULT_FETCH_RETRY_LIMIT = 5;
-  public static final String BULK_API_USE_QUERY_ALL = "salesforce.bulkApiUseQueryAll";
 
   // bulk api retry sleep duration for avoid resource consuming peak.
   public static final String RETRY_EXCEED_QUOTA_INTERVAL = "salesforce.retry.exceedQuotaInterval";
@@ -39,10 +37,6 @@ public final class SalesforceConfigurationKeys {
   public static final String BULK_TEST_JOB_ID = "salesforce.bulk.testJobId";
   public static final String BULK_TEST_BATCH_ID_LIST = "salesforce.bulk.testBatchIds";
   public static final String SALESFORCE_PARTITION_TYPE = "salesforce.partitionType";
-  public static final String PARTITION_PK_CHUNKING_SIZE = "salesforce.partition.pkChunkingSize";
   public static final String PK_CHUNKING_JOB_ID = "__salesforce.job.id"; // don't use in ini config
   public static final String PK_CHUNKING_BATCH_RESULT_ID_PAIRS = "__salesforce.batch.result.id.pairs"; // don't use in ini config
-  public static final int MAX_PK_CHUNKING_SIZE = 250_000; // this number is from SFDC's doc - https://tinyurl.com/ycjvgwv2
-  public static final int MIN_PK_CHUNKING_SIZE = 20_000;
-  public static final int DEFAULT_PK_CHUNKING_SIZE = 250_000; // default to max for saving request quota
 }
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceExtractor.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceExtractor.java
index 2a5d8bc..868ed54 100644
--- a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceExtractor.java
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceExtractor.java
@@ -100,8 +100,6 @@ public class SalesforceExtractor extends RestApiExtractor {
   private static final String SALESFORCE_SOAP_SERVICE = "/services/Soap/u";
   private static final Gson GSON = new Gson();
   private static final int MAX_RETRY_INTERVAL_SECS = 600;
-  private static final String FETCH_RETRY_LIMIT_KEY = "salesforce.fetchRetryLimit";
-  private static final boolean DEFAULT_BULK_API_USE_QUERY_ALL = false;
 
   private boolean pullStatus = true;
   private String nextUrl;
@@ -120,20 +118,20 @@ public class SalesforceExtractor extends RestApiExtractor {
   private final long retryExceedQuotaInterval;
 
   private final boolean bulkApiUseQueryAll;
+  private SfConfig conf;
 
 
   public SalesforceExtractor(WorkUnitState state) {
     super(state);
+    conf = new SfConfig(state.getProperties());
 
     this.sfConnector = (SalesforceConnector) this.connector;
-    this.pkChunkingSize =
-        Math.max(MIN_PK_CHUNKING_SIZE,
-            Math.min(MAX_PK_CHUNKING_SIZE, workUnitState.getPropAsInt(PARTITION_PK_CHUNKING_SIZE, DEFAULT_PK_CHUNKING_SIZE)));
+    this.pkChunkingSize = conf.pkChunkingSize;
 
-    this.bulkApiUseQueryAll = workUnitState.getPropAsBoolean(BULK_API_USE_QUERY_ALL, DEFAULT_BULK_API_USE_QUERY_ALL);
-    this.retryLimit = workUnitState.getPropAsInt(FETCH_RETRY_LIMIT_KEY, DEFAULT_FETCH_RETRY_LIMIT);
     this.retryInterval = workUnitState.getPropAsLong(RETRY_INTERVAL, RETRY_INTERVAL_DEFAULT);
     this.retryExceedQuotaInterval = workUnitState.getPropAsLong(RETRY_EXCEED_QUOTA_INTERVAL, RETRY_EXCEED_QUOTA_INTERVAL_DEFAULT);
+    this.bulkApiUseQueryAll = conf.bulkApiUseQueryAll;
+    this.retryLimit = conf.fetchRetryLimit;
   }
 
   @Override
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SfConfig.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SfConfig.java
new file mode 100644
index 0000000..4377237
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SfConfig.java
@@ -0,0 +1,39 @@
+/*
+ * 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.gobblin.salesforce;
+
+import java.util.Properties;
+import org.apache.gobblin.typedconfig.Default;
+import org.apache.gobblin.typedconfig.Key;
+import org.apache.gobblin.typedconfig.compiletime.IntRange;
+
+
+public class SfConfig extends QueryBasedSourceConfig {
+  public SfConfig(Properties prop) {
+    super(prop);
+  }
+
+  @Key("salesforce.partition.pkChunkingSize")@Default("250000")@IntRange({20_000, 250_000})
+  public int pkChunkingSize;
+
+  @Key("salesforce.bulkApiUseQueryAll")@Default("false")
+  public boolean bulkApiUseQueryAll;
+
+  @Key("salesforce.fetchRetryLimit")@Default("5")
+  public int fetchRetryLimit;
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Alias.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Alias.java
new file mode 100644
index 0000000..1a0110e
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Alias.java
@@ -0,0 +1,30 @@
+/*
+ * 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.gobblin.typedconfig;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+
+@Target({ElementType.FIELD, ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface Alias {
+  String value();
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/ConstraintUtil.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/ConstraintUtil.java
new file mode 100644
index 0000000..af4b57f
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/ConstraintUtil.java
@@ -0,0 +1,135 @@
+/*
+ * 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.gobblin.typedconfig;
+
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.gobblin.typedconfig.compiletime.EnumOptions;
+import org.apache.gobblin.typedconfig.compiletime.IntRange;
+import org.apache.gobblin.typedconfig.compiletime.LongRange;
+import org.apache.gobblin.typedconfig.compiletime.StringRegex;
+
+
+/**
+ * Util class for handling constraint annotation
+ * supports: IntRange, LongRange, EnumOption, StringRegex
+ */
+@Slf4j
+public class ConstraintUtil {
+  private ConstraintUtil() {
+  }
+
+  public static Object constraint(Field field, Object value) {
+    Class type = field.getType();
+    if (type.isEnum()) {
+      return getEnumValue(field, value);
+    }
+    switch (type.getName()) {
+      case "int": case "java.lang.Integer":
+        return getIntValue(field, value);
+      case "long": case "java.lang.Long":
+        return getLongValue(field, value);
+      case "java.lang.String":
+        return getStringValue(field, value);
+      case "boolean": case "java.lang.Boolean":
+        return getBooleanValue(field, value);
+      case "java.util.Date":
+        return getDateValue(field, value);
+      default:
+        throw new RuntimeException("not supported the return type: " + type.getName());
+    }
+  }
+
+  static private Object getIntValue(Field field, Object value) {
+    IntRange intRange = field.getAnnotation(IntRange.class);
+    if (intRange == null) {
+      return value;
+    }
+    int[] range = intRange.value();
+    int intValue = Integer.parseInt(value.toString());
+    if (range.length != 2) {
+      throw new RuntimeException(String.format("Field [%s]: Long range is invalid.", field.getName()));
+    }
+    if (intValue >= range[0] && intValue <= range[1]) {
+      return value;
+    } else {
+      throw new RuntimeException(
+          String.format("Field [%s]: value [%s] is out of range [%s, %s].", field.getName(), value, range[0], range[1])
+      );
+    }
+  }
+  static private Object getLongValue(Field field, Object value) {
+    LongRange longRange = field.getAnnotation(LongRange.class);
+    long[] range = longRange.value();
+    if (range == null) {
+      return value;
+    }
+    long longValue = Long.parseLong(value.toString());
+    if (range.length != 2) {
+      throw new RuntimeException(String.format("Field [%s]: Long range is invalid.", field.getName()));
+    }
+    if (longValue > range[0] && longValue < range[1]) {
+      return value;
+    } else {
+      throw new RuntimeException(
+          String.format("Field [%s]: value [%s] is out of range [%s, %s].", field.getName(), value, range[0], range[1])
+      );
+    }
+  }
+
+  static private Object getStringValue(Field field, Object value) {
+    StringRegex stringRegex = field.getAnnotation(StringRegex.class);
+    if (stringRegex == null) {
+      return value;
+    }
+    String regex = stringRegex.value();
+    if (regex == null) {
+      return value;
+    }
+    boolean isMatching = value.toString().matches(regex);
+    if (isMatching) {
+      return value;
+    } else {
+      throw new RuntimeException(String.format("[%s] is not matching pattern [%s]", value, regex));
+    }
+  }
+
+  static private Object getBooleanValue(Field field, Object value) {
+    return value; // there is no restriction for boolean value.
+  }
+
+  static private Object getDateValue(Field field, Object value) {
+    return value; // there is no restriction for Date value.
+  }
+
+  static private Object getEnumValue(Field field, Object value) {
+    EnumOptions enumOptions = field.getAnnotation(EnumOptions.class);
+    if (enumOptions == null) {
+      return value;
+    }
+    List<String> options = Arrays.asList(enumOptions.value());
+    if (options.indexOf(value) >= 0) {
+      return value;
+    } else {
+      throw new RuntimeException(String.format("Enum [%s] is not allowed.", value));
+    }
+  }
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Default.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Default.java
new file mode 100644
index 0000000..cd12fdf
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Default.java
@@ -0,0 +1,30 @@
+/*
+ * 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.gobblin.typedconfig;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+
+@Target({ElementType.FIELD, ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface Default {
+  String value();
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Key.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Key.java
new file mode 100644
index 0000000..cb43249
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/Key.java
@@ -0,0 +1,31 @@
+/*
+ * 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.gobblin.typedconfig;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+
+@Target({ElementType.FIELD, ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface Key {
+  String value();
+}
+
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/TypedConfig.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/TypedConfig.java
new file mode 100644
index 0000000..cd70794
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/TypedConfig.java
@@ -0,0 +1,127 @@
+/*
+ * 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.gobblin.typedconfig;
+
+import java.lang.reflect.Field;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Properties;
+import lombok.SneakyThrows;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean;
+
+
+public class TypedConfig {
+
+  public TypedConfig(Properties prop) {
+    if (prop != null) {
+      fulfillFields(this.getClass(), prop);
+    }
+  }
+
+  @SneakyThrows
+  private void fulfillFields(Class<? extends TypedConfig> clazz, Properties prop) {
+    if (!clazz.equals(TypedConfig.class)) {
+      this.fulfillFields((Class<? extends TypedConfig>) clazz.getSuperclass(), prop);
+    }
+    for (Field field : clazz.getDeclaredFields()) {
+      if (field.getAnnotations().length == 0) {
+        continue;
+      }
+      Object defaultValue = pickupDefaultValue(field); // get default value which is in config class
+      Object configValue = pickupValueByKey(field, prop); // get ini file config value by key
+      if (configValue == null) {
+        configValue = pickupValueByAlias(field, prop); // by alias (2nd key)
+      }
+      if (configValue == null) {
+        configValue = defaultValue;
+      }
+      if (configValue != null) {
+        configValue = ConstraintUtil.constraint(field, configValue);
+        field.set(this, convert(configValue, field.getType()));
+      }
+    }
+  }
+
+  private Object pickupDefaultValue(Field field) {
+    Default defaultAnn = field.getAnnotation(Default.class);
+    if (defaultAnn == null) {
+      return null;
+    }
+    return defaultAnn.value(); // the value was put in source code instead of ini file
+  }
+
+  private Object pickupValueByAlias(Field field, Properties prop) {
+    Alias alias = field.getAnnotation(Alias.class);
+    if (alias == null) {
+      return null;
+    }
+    return prop.get(alias.value()); // get ini config value by alias(2nd key)
+  }
+
+  private Object pickupValueByKey(Field field,  Properties prop) {
+    Key key = field.getAnnotation(Key.class);
+    if (key == null) {
+      return null;
+    }
+    return prop.get(key.value()); // get ini config value by key
+  }
+
+  private Object convert(Object value, Class targetClazz) {
+    if (value == null) {
+      return null;
+    }
+    BeanUtilsBean beanUtilsBean = new BeanUtilsBean(new ConvertUtilsBean() {
+      @SneakyThrows
+      @Override
+      public Object convert(Object value, Class clazz) {
+        if (clazz.isEnum()) {
+          return Enum.valueOf(clazz, (String) value);
+        } else if (clazz == Date.class) {
+          String dateStr = ((String) value).replaceAll("-| |:", ""); // date format: 1, 2020-01-02 03:04:59, 2, 20200102030459
+          dateStr = String.format("%-14s", dateStr).replaceAll(" ", "0");
+          Date date = new SimpleDateFormat("yyyyMMddHHmmss").parse(dateStr);
+          return date;
+        } else {
+          return super.convert(value, clazz);
+        }
+      }
+    });
+    return beanUtilsBean.getConvertUtils().convert(value, targetClazz);
+  }
+
+  /**
+   * convert data to property
+   */
+  @SneakyThrows
+  public Properties toProp() {
+    Properties prop = new Properties();
+    for (Field field : this.getClass().getDeclaredFields()) {
+      Key keyAnn = field.getAnnotation(Key.class);
+      if (keyAnn == null) {
+        continue;
+      }
+      Object configValue = field.get(this);
+      if (configValue == null) {
+        continue;
+      }
+      prop.put(keyAnn.value(), configValue.toString());
+    }
+    return prop;
+  }
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/EnumOptions.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/EnumOptions.java
new file mode 100644
index 0000000..925ee20
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/EnumOptions.java
@@ -0,0 +1,30 @@
+/*
+ * 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.gobblin.typedconfig.compiletime;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+
+@Target({ElementType.FIELD, ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface EnumOptions {
+  String[] value();
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/IntRange.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/IntRange.java
new file mode 100644
index 0000000..9ba6077
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/IntRange.java
@@ -0,0 +1,30 @@
+/*
+ * 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.gobblin.typedconfig.compiletime;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+
+@Target({ElementType.FIELD, ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface IntRange {
+  int[] value();
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/LongRange.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/LongRange.java
new file mode 100644
index 0000000..0362b88
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/LongRange.java
@@ -0,0 +1,30 @@
+/*
+ * 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.gobblin.typedconfig.compiletime;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+
+@Target({ElementType.FIELD, ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface LongRange {
+  long[] value();
+}
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/StringRegex.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/StringRegex.java
new file mode 100644
index 0000000..1a8bd97
--- /dev/null
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/typedconfig/compiletime/StringRegex.java
@@ -0,0 +1,30 @@
+/*
+ * 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.gobblin.typedconfig.compiletime;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+
+@Target({ElementType.FIELD, ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface StringRegex {
+  String value();
+}