You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gobblin.apache.org by GitBox <gi...@apache.org> on 2023/01/05 17:19:38 UTC

[GitHub] [gobblin] Will-Lo commented on a diff in pull request #3614: Add error reporting when attempting to resolve flow configs

Will-Lo commented on code in PR #3614:
URL: https://github.com/apache/gobblin/pull/3614#discussion_r1062687327


##########
gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigV2ResourceLocalHandler.java:
##########
@@ -111,17 +117,46 @@ public CreateKVResponse createFlowConfig(FlowConfig flowConfig, boolean triggerL
 
   private String getErrorMessage(FlowSpec flowSpec) {
     StringBuilder message = new StringBuilder("Flow was not compiled successfully.");
+    Hashtable<String, ArrayList<String>> allErrors = new Hashtable<>();
+
     if (!flowSpec.getCompilationErrors().isEmpty()) {
       message.append(" Compilation errors encountered (Sorted by relevance): ");
       FlowSpec.CompilationError[] errors = flowSpec.getCompilationErrors().stream().distinct().toArray(FlowSpec.CompilationError[]::new);
       Arrays.sort(errors, Comparator.comparingInt(c -> ((FlowSpec.CompilationError)c).errorPriority));
-      int errorId = 0;
+      int errorIdSingleHop = 1;
+      int errorIdMultiHop = 1;
+
+      ArrayList<String> singleHopErrors = new ArrayList<>();
+      ArrayList<String> multiHopErrors = new ArrayList<>();
+
       for (FlowSpec.CompilationError error: errors) {
-        message.append("\n").append(String.format("ERROR[%s]", errorId)).append(error.errorMessage);
-        errorId++;
+        if (error.errorPriority == 0) {
+          singleHopErrors.add(String.format("ERROR %s of single hop: ", errorIdSingleHop) + error.errorMessage.replace("\n", " ").replace("\t", ""));
+          errorIdSingleHop++;
+        }
+        else {
+          multiHopErrors.add(String.format("ERROR %s of multi hop: ", errorIdMultiHop) + error.errorMessage.replace("\n", " ").replace("\t", ""));
+          errorIdMultiHop++;
+        }
       }
+
+      allErrors.put("singleHopErrors", singleHopErrors);
+      allErrors.put("multiHopErrors", multiHopErrors);
+    }
+
+
+    allErrors.put("message", new ArrayList<>(Collections.singletonList(message.toString())));
+    ObjectMapper mapper = new ObjectMapper();
+
+    try {
+      return mapper.writeValueAsString(allErrors);
+    }
+    catch (JsonProcessingException e) {
+      log.error("Flow Spec Errored on Json Processing");
+      log.error(flowSpec.toString());
+      e.printStackTrace();

Review Comment:
   You should combine all of these into one log statement, makes it easier to parse through log tooling. 
   e.g. `log.error("Flow Spec {} errored on Json processing", flowSpec.toString(), e);`
   



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/BaseDatasetDescriptor.java:
##########
@@ -57,37 +62,48 @@ public BaseDatasetDescriptor(Config config) throws IOException {
     this.isRetentionApplied = ConfigUtils.getBoolean(config, DatasetDescriptorConfigKeys.IS_RETENTION_APPLIED_KEY, false);
     this.description = ConfigUtils.getString(config, DatasetDescriptorConfigKeys.DESCRIPTION_KEY, "");
     this.rawConfig = config.withFallback(this.formatConfig.getRawConfig()).withFallback(DEFAULT_FALLBACK);
+    this.isInputDataset = ConfigUtils.getBoolean(config, DatasetDescriptorConfigKeys.IS_INPUT_DATASET, false);
   }
 
   /**
    * {@inheritDoc}
    */
-  protected abstract boolean isPathContaining(DatasetDescriptor other);
+  protected abstract ArrayList<String> isPathContaining(DatasetDescriptor other);
 
   /**
    * @return true if this {@link DatasetDescriptor} contains the other {@link DatasetDescriptor} i.e. the
    * datasets described by this {@link DatasetDescriptor} is a subset of the datasets described by the other
    * {@link DatasetDescriptor}. This operation is non-commutative.
-   * @param other
+   * @param userFlowConfig

Review Comment:
   Mention what this parameter means in the javadoc



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/FormatConfig.java:
##########
@@ -67,24 +70,40 @@ public FormatConfig(Config config) throws IOException {
         .empty()));
     this.rawConfig = config.withFallback(this.encryptionConfig.getRawConfig().atPath(DatasetDescriptorConfigKeys.ENCYPTION_PREFIX)).
         withFallback(DEFAULT_FALLBACK);
+    this.isInputDataset = ConfigUtils.getBoolean(config, DatasetDescriptorConfigKeys.IS_INPUT_DATASET, false);
   }
 
-  public boolean contains(FormatConfig other) {
-    return containsFormat(other.getFormat()) && containsCodec(other.getCodecType())
-        && containsEncryptionConfig(other.getEncryptionConfig());
+  public ArrayList<String> contains(FormatConfig userFlowConfig) {
+    ArrayList<String> errors = new ArrayList<>();
+    errors.addAll(containsFormat(userFlowConfig.getFormat(), userFlowConfig.getIsInputDataset()));
+    errors.addAll(containsCodec(userFlowConfig.getCodecType(), userFlowConfig.getIsInputDataset()));
+    errors.addAll(containsEncryptionConfig(userFlowConfig.getEncryptionConfig()));
+    return errors;
   }
 
-  private boolean containsFormat(String otherFormat) {
-    return DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getFormat())
-        || (this.getFormat().equalsIgnoreCase(otherFormat));
+  private ArrayList<String> containsFormat(String userFlowConfigFormat, Boolean inputDataset) {
+    ArrayList<String> errors = new ArrayList<>();
+    String datasetDescriptorPrefix = inputDataset ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getFormat())
+        && (!this.getFormat().equalsIgnoreCase(userFlowConfigFormat))) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.FORMAT_KEY + " is mismatched. User input: '" + userFlowConfigFormat
+          + "'. Expected value: '" + this.getFormat() + "'.");
+    }
+    return errors;
   }
 
-  private boolean containsCodec(String otherCodecType) {
-    return DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getCodecType())
-        || (this.getCodecType().equalsIgnoreCase(otherCodecType));
+  private ArrayList<String> containsCodec(String userFlowConfigCodecType, Boolean inputDataset) {
+    ArrayList<String> errors = new ArrayList<>();
+    String datasetDescriptorPrefix = inputDataset ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getCodecType())
+        && (!this.getCodecType().equalsIgnoreCase(userFlowConfigCodecType))) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.CODEC_KEY + " is mismatched. User input: '" + userFlowConfigCodecType
+          + "'. Expected value: '" + this.getCodecType() + "'.");
+    }
+    return errors;
   }

Review Comment:
   template these



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/BaseDatasetDescriptor.java:
##########
@@ -57,37 +62,48 @@ public BaseDatasetDescriptor(Config config) throws IOException {
     this.isRetentionApplied = ConfigUtils.getBoolean(config, DatasetDescriptorConfigKeys.IS_RETENTION_APPLIED_KEY, false);
     this.description = ConfigUtils.getString(config, DatasetDescriptorConfigKeys.DESCRIPTION_KEY, "");
     this.rawConfig = config.withFallback(this.formatConfig.getRawConfig()).withFallback(DEFAULT_FALLBACK);
+    this.isInputDataset = ConfigUtils.getBoolean(config, DatasetDescriptorConfigKeys.IS_INPUT_DATASET, false);
   }
 
   /**
    * {@inheritDoc}
    */
-  protected abstract boolean isPathContaining(DatasetDescriptor other);
+  protected abstract ArrayList<String> isPathContaining(DatasetDescriptor other);
 
   /**
    * @return true if this {@link DatasetDescriptor} contains the other {@link DatasetDescriptor} i.e. the
    * datasets described by this {@link DatasetDescriptor} is a subset of the datasets described by the other
    * {@link DatasetDescriptor}. This operation is non-commutative.
-   * @param other
+   * @param userFlowConfig
    */
   @Override
-  public boolean contains(DatasetDescriptor other) {
-    if (this == other) {
-      return true;
+  public ArrayList<String> contains(DatasetDescriptor userFlowConfig) {
+    ArrayList<String> errors = new ArrayList<>();
+    String datasetDescriptorPrefix = userFlowConfig.getIsInputDataset() ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    if (this == userFlowConfig) {
+      return errors;
     }
 
-    if (other == null || !getClass().equals(other.getClass())) {
-      return false;
+    if (!getClass().equals(userFlowConfig.getClass())) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.CLASS_KEY + " is mismatched. User input: '" + userFlowConfig.getClass()

Review Comment:
   Consider putting these template messages in the base class and have the other dataset descriptors leverage it.



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/FSDatasetPartitionConfig.java:
##########
@@ -128,13 +133,21 @@ private void validatePartitionConfig(String partitionType, String partitionPatte
     }
   }
 
-  public boolean contains(FSDatasetPartitionConfig other) {
-    if (other == null) {
-      return false;
+  public ArrayList<String> contains(FSDatasetPartitionConfig userFlowConfig) {
+    String datasetDescriptorPrefix = userFlowConfig.getIsInputDataset() ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    ArrayList<String> errors = new ArrayList<>();
+
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getPartitionType())
+        && !this.getPartitionType().equalsIgnoreCase(userFlowConfig.getPartitionType())) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.PARTITION_PREFIX + "." + DatasetDescriptorConfigKeys.PARTITION_TYPE_KEY + " is mismatched. User input: '" + userFlowConfig.getPartitionType()
+          + "'. Expected value: '" + this.getPartitionType() + "'.");
+    }
+
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(getPartitionPattern())
+        && !this.getPartitionPattern().equalsIgnoreCase(userFlowConfig.getPartitionPattern())) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.PARTITION_PREFIX + "." + DatasetDescriptorConfigKeys.PARTITION_PATTERN_KEY + " is mismatched. User input: '" + userFlowConfig.getPartitionPattern()
+          + "'. Expected value: '" + this.getPartitionPattern() + "'.");

Review Comment:
   These can be templated



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/BaseDatasetDescriptor.java:
##########
@@ -57,37 +62,48 @@ public BaseDatasetDescriptor(Config config) throws IOException {
     this.isRetentionApplied = ConfigUtils.getBoolean(config, DatasetDescriptorConfigKeys.IS_RETENTION_APPLIED_KEY, false);
     this.description = ConfigUtils.getString(config, DatasetDescriptorConfigKeys.DESCRIPTION_KEY, "");
     this.rawConfig = config.withFallback(this.formatConfig.getRawConfig()).withFallback(DEFAULT_FALLBACK);
+    this.isInputDataset = ConfigUtils.getBoolean(config, DatasetDescriptorConfigKeys.IS_INPUT_DATASET, false);
   }
 
   /**
    * {@inheritDoc}
    */
-  protected abstract boolean isPathContaining(DatasetDescriptor other);
+  protected abstract ArrayList<String> isPathContaining(DatasetDescriptor other);
 
   /**
    * @return true if this {@link DatasetDescriptor} contains the other {@link DatasetDescriptor} i.e. the
    * datasets described by this {@link DatasetDescriptor} is a subset of the datasets described by the other
    * {@link DatasetDescriptor}. This operation is non-commutative.
-   * @param other
+   * @param userFlowConfig
    */
   @Override
-  public boolean contains(DatasetDescriptor other) {
-    if (this == other) {
-      return true;
+  public ArrayList<String> contains(DatasetDescriptor userFlowConfig) {
+    ArrayList<String> errors = new ArrayList<>();
+    String datasetDescriptorPrefix = userFlowConfig.getIsInputDataset() ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    if (this == userFlowConfig) {
+      return errors;
     }
 
-    if (other == null || !getClass().equals(other.getClass())) {
-      return false;
+    if (!getClass().equals(userFlowConfig.getClass())) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.CLASS_KEY + " is mismatched. User input: '" + userFlowConfig.getClass()

Review Comment:
   Errors like these may benefit from `String.format()` function, will help it become more readable. Additionally you can actually have code where you have two constant static strings:
   ```
   static DATASET_DESCRIPTOR_KEY_MISMATCH_ERROR_TEMPLATE = "%s.%s is mismatched. User input: %s. Expected value %s"
   ```
   and you can add your errors in a consistent way like this:
   ```
   errors.add(String.format(DATASET_DESCRIPTOR_KEY_MISMATCH_ERROR_TEMPLATE, datasetDescriptorPrefix, DatasetDescriptorConfigKeys.CLASS_KEY, userFlowConfig.getClass(), this.getClass());
   ```



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/FSDatasetDescriptor.java:
##########
@@ -113,41 +119,61 @@ protected boolean isPathContaining(DatasetDescriptor other) {
    * accepted by the other {@link DatasetDescriptor}. If the path description of the other {@link DatasetDescriptor}
    * is a glob pattern, we return false.
    *
-   * @param otherPath a glob pattern that describes a set of paths.
+   * @param userFlowConfigPath a glob pattern that describes a set of paths.
    * @return true if the glob pattern described by the otherPath matches the path in this {@link DatasetDescriptor}.
    */
-  private boolean isPathContaining(String otherPath) {
-    if (otherPath == null) {
-      return false;
+  private ArrayList<String> isPathContaining(String userFlowConfigPath, Boolean inputDataset) {
+    String datasetDescriptorPrefix = inputDataset ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    ArrayList<String> errors = new ArrayList<>();
+    if (userFlowConfigPath == null) {
+      errors.add(datasetDescriptorPrefix + DatasetDescriptorConfigKeys.PATH_KEY + " is empty. Expected value: " + this.getPath());
+      return errors;
     }
     if (DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equals(this.getPath())) {
-      return true;
+      return errors;
     }
 
-    if (PathUtils.isGlob(new Path(otherPath))) {
-      return false;
+    if (PathUtils.isGlob(new Path(userFlowConfigPath))) {
+      errors.add(datasetDescriptorPrefix + DatasetDescriptorConfigKeys.PATH_KEY + " is a glob pattern. User input: '" + userFlowConfigPath
+          + "'. Expected input is not of a glob pattern.");
+      return errors;
     }
 
     GlobPattern globPattern = new GlobPattern(this.getPath());
-    return globPattern.matches(otherPath);
+
+    if (!globPattern.matches(userFlowConfigPath)) {
+      errors.add(datasetDescriptorPrefix + ".globPattern is mismatched. User input: '" + userFlowConfigPath
+          + "'. Expected value path of: " + this.getPath() + " and globPattern of '" + globPattern + "'.");
+    }
+    return errors;
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public boolean contains(DatasetDescriptor o) {
-    if (!super.contains(o)) {
-      return false;
+  public ArrayList<String> contains(DatasetDescriptor userFlowConfigDatasetDescriptor) {
+    String datasetDescriptorPrefix = userFlowConfigDatasetDescriptor.getIsInputDataset() ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    ArrayList<String> errors = new ArrayList<>();
+    if (super.contains(userFlowConfigDatasetDescriptor).size() != 0) {
+      return super.contains(userFlowConfigDatasetDescriptor);
     }
 
-    FSDatasetDescriptor other = (FSDatasetDescriptor) o;
+    FSDatasetDescriptor userFlowConfig = (FSDatasetDescriptor) userFlowConfigDatasetDescriptor;
 
-    if ((this.isCompacted() != other.isCompacted()) ||
-        (this.isCompactedAndDeduped() != other.isCompactedAndDeduped())) {
-      return false;
+    if ((this.isCompacted() != userFlowConfig.isCompacted()) ||
+        (this.isCompactedAndDeduped() != userFlowConfig.isCompactedAndDeduped())) {
+      if (this.isCompacted() != userFlowConfig.isCompacted()) {
+        errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.IS_COMPACTED_KEY + " is mismatched. User input: '" + userFlowConfig.isCompacted()
+            + "'. Expected value: '" + this.isCompacted() + "'.");
+      }
+      else {
+        errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.IS_COMPACTED_AND_DEDUPED_KEY + " is mismatched. User input: '" + userFlowConfig.isCompactedAndDeduped()
+            + "'. Expected value: '" + this.isCompactedAndDeduped() + "'.");

Review Comment:
   These can be templated



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/EncryptionConfig.java:
##########
@@ -124,26 +128,46 @@ private void validate(String encryptionLevel, String encryptedFields) throws IOE
     return;
   }
 
-  public boolean contains(EncryptionConfig other) {
-    if (other == null) {
-      return false;
+  public ArrayList<String> contains(EncryptionConfig userFlowConfig) {
+    String datasetDescriptorPrefix = userFlowConfig.getIsInputDataset() ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    ArrayList<String> errors = new ArrayList<>();
+
+    String userFlowEncryptionAlgorithm = userFlowConfig.getEncryptionAlgorithm();
+    String userFlowKeystoreType = userFlowConfig.getKeystoreType();
+    String userFlowKeystoreEncoding = userFlowConfig.getKeystoreEncoding();
+    String userFlowEncryptionLevel = userFlowConfig.getEncryptionLevel();
+    String userFlowEncryptedFields = userFlowConfig.getEncryptedFields();
+
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getEncryptionAlgorithm())
+        && !this.encryptionAlgorithm.equalsIgnoreCase(userFlowEncryptionAlgorithm)) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.ENCRYPTION_ALGORITHM_KEY + " is mismatched. User input: '" + userFlowEncryptionAlgorithm
+          + "'. Expected value: '" + this.getEncryptionAlgorithm() + "'.");
+    }
+
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getKeystoreType())
+        && !this.keystoreType.equalsIgnoreCase(userFlowKeystoreType)) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.ENCRYPTION_KEYSTORE_TYPE_KEY + " is mismatched. User input: '" + userFlowKeystoreType
+          + "'. Expected value: '" + this.getKeystoreType() + "'.");
+    }
+
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getKeystoreEncoding())
+        && !this.keystoreEncoding.equalsIgnoreCase(userFlowKeystoreEncoding)) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.ENCRYPTION_KEYSTORE_ENCODING_KEY + " is mismatched. User input: '" + userFlowKeystoreEncoding
+          + "'. Expected value: " + this.getKeystoreEncoding() + "'.");
+    }
+
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getEncryptionLevel())
+        && !this.encryptionLevel.equalsIgnoreCase(userFlowEncryptionLevel)) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.ENCRYPTION_LEVEL_KEY + " is mismatched. User input: '" + userFlowEncryptionLevel
+          + "'. Expected value: '" + this.getEncryptionLevel()  + "'.");
+    }
+
+    if (!DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equalsIgnoreCase(this.getEncryptedFields())
+        && !this.encryptedFields.equalsIgnoreCase(userFlowEncryptedFields)) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.ENCRYPTED_FIELDS + " is mismatched. User input: '" + userFlowEncryptedFields
+          + "'. Expected value: '" + this.getEncryptedFields() + ".");

Review Comment:
   These can use the templated error message as well



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/FSDatasetDescriptor.java:
##########
@@ -113,41 +119,61 @@ protected boolean isPathContaining(DatasetDescriptor other) {
    * accepted by the other {@link DatasetDescriptor}. If the path description of the other {@link DatasetDescriptor}
    * is a glob pattern, we return false.
    *
-   * @param otherPath a glob pattern that describes a set of paths.
+   * @param userFlowConfigPath a glob pattern that describes a set of paths.
    * @return true if the glob pattern described by the otherPath matches the path in this {@link DatasetDescriptor}.
    */
-  private boolean isPathContaining(String otherPath) {
-    if (otherPath == null) {
-      return false;
+  private ArrayList<String> isPathContaining(String userFlowConfigPath, Boolean inputDataset) {
+    String datasetDescriptorPrefix = inputDataset ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    ArrayList<String> errors = new ArrayList<>();
+    if (userFlowConfigPath == null) {
+      errors.add(datasetDescriptorPrefix + DatasetDescriptorConfigKeys.PATH_KEY + " is empty. Expected value: " + this.getPath());
+      return errors;
     }
     if (DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equals(this.getPath())) {
-      return true;
+      return errors;
     }
 
-    if (PathUtils.isGlob(new Path(otherPath))) {
-      return false;
+    if (PathUtils.isGlob(new Path(userFlowConfigPath))) {
+      errors.add(datasetDescriptorPrefix + DatasetDescriptorConfigKeys.PATH_KEY + " is a glob pattern. User input: '" + userFlowConfigPath
+          + "'. Expected input is not of a glob pattern.");
+      return errors;
     }
 
     GlobPattern globPattern = new GlobPattern(this.getPath());
-    return globPattern.matches(otherPath);
+
+    if (!globPattern.matches(userFlowConfigPath)) {
+      errors.add(datasetDescriptorPrefix + ".globPattern is mismatched. User input: '" + userFlowConfigPath
+          + "'. Expected value path of: " + this.getPath() + " and globPattern of '" + globPattern + "'.");

Review Comment:
   I don't think .globPattern is a key, it just describes a `DatasetDescriptorConfigKeys.PATH_KEY` that is of glob format e.g. `/path/*`
   
   Semantically the check here checks that if the conf defines that your path needs to be of a certain glob e.g. `/path/*` your input path needs to be contained in said glob.
   
   So the error should look like: DatasetDescriptorPrefix + DatasetDescriptorConfigKeys.PATH_KEY + " user value <userFlowConfigPath> is not contained within the glob of <this.getPath()>. " with proper formatting/templates
   
   



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/FSVolumeDatasetDescriptor.java:
##########
@@ -30,27 +31,34 @@
  * An implementation of {@link FSVolumeDatasetDescriptor} with fs.uri specified.
  */
 @Alpha
-@ToString(callSuper = true, exclude = {"rawConfig"})
-@EqualsAndHashCode(callSuper = true, exclude = {"rawConfig"})
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
 public class FSVolumeDatasetDescriptor extends FSDatasetDescriptor{
   @Getter
   private final String fsUri;
 
   public FSVolumeDatasetDescriptor(Config config) throws IOException {
     super(config);
     this.fsUri = ConfigUtils.getString(config, DatasetDescriptorConfigKeys.FS_URI_KEY, DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY);
+    this.isInputDataset = ConfigUtils.getBoolean(config, DatasetDescriptorConfigKeys.IS_INPUT_DATASET, false);
   }
 
   @Override
-  public boolean contains(DatasetDescriptor o) {
-    if (!super.contains(o)) {
-      return false;
+  public ArrayList<String> contains(DatasetDescriptor userFlowConfig) {
+    String datasetDescriptorPrefix = userFlowConfig.getIsInputDataset() ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    ArrayList<String> errors = new ArrayList<>();
+    if (super.contains(userFlowConfig).size() != 0) {
+      return super.contains(userFlowConfig);
     }
 
-    FSVolumeDatasetDescriptor other = (FSVolumeDatasetDescriptor) o;
+    FSVolumeDatasetDescriptor other = (FSVolumeDatasetDescriptor) userFlowConfig;
 
-    return DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equals(this.getFsUri()) || this.getFsUri()
-        .equals(other.getFsUri());
+    if (!(DatasetDescriptorConfigKeys.DATASET_DESCRIPTOR_CONFIG_ANY.equals(this.getFsUri()) || this.getFsUri()
+        .equals(other.getFsUri()))) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.FS_URI_KEY + " is mismatched. User input: '" + ((FSVolumeDatasetDescriptor) userFlowConfig).getFsUri()
+          + "'. Expected value: '" + this.getFsUri() + "'.");
+    }
+    return errors;

Review Comment:
   Template this



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/dataset/HiveDatasetDescriptor.java:
##########
@@ -99,35 +102,44 @@ protected boolean isPlatformValid() {
   }
 
   @Override
-  protected boolean isPathContaining(DatasetDescriptor other) {
-    String otherPath = other.getPath();
+  protected ArrayList<String> isPathContaining(DatasetDescriptor userFlowConfig) {
+    String datasetDescriptorPrefix = userFlowConfig.getIsInputDataset() ? DatasetDescriptorConfigKeys.FLOW_INPUT_DATASET_DESCRIPTOR_PREFIX : DatasetDescriptorConfigKeys.FLOW_OUTPUT_DATASET_DESCRIPTOR_PREFIX;
+    ArrayList<String> errors = new ArrayList<>();
+    String otherPath = userFlowConfig.getPath();
     if (otherPath == null) {
-      return false;
+      errors.add(datasetDescriptorPrefix + DatasetDescriptorConfigKeys.PATH_KEY + " is missing"
+          + ". Expected value: '" + this.getPath() +  ".");
+      return errors;
     }
 
-    if (this.isPartitioned != ((HiveDatasetDescriptor) other).isPartitioned) {
-      return false;
+    if (this.isPartitioned != ((HiveDatasetDescriptor) userFlowConfig).isPartitioned) {
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.PARTITION_PREFIX + "." + DatasetDescriptorConfigKeys.PARTITION_TYPE_KEY + " is mismatched. User input: '" + ((HiveDatasetDescriptor) userFlowConfig).isPartitioned
+          + "'. Expected value: '" + this.isPartitioned + ".");
     }
 
     //Extract the dbName and tableName from otherPath
     List<String> parts = Splitter.on(SEPARATION_CHAR).splitToList(otherPath);
     if (parts.size() != 2) {
-      return false;
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.PATH_KEY + " is mismatched. User input: '" + otherPath + "' is not splittable"
+          + ". Expected separation character: '" + SEPARATION_CHAR +  "'.");
+      return errors;
     }
 
     String otherDbName = parts.get(0);
     String otherTableNames = parts.get(1);
 
     if (!this.whitelistBlacklist.acceptDb(otherDbName)) {
-      return false;
+      errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.DATABASE_KEY + " is mismatched. User input: '" + otherDbName + "' is in the blacklist"
+          + ".");
     }
 
     List<String> otherTables = Splitter.on(",").splitToList(otherTableNames);
     for (String otherTable : otherTables) {
       if (!this.whitelistBlacklist.acceptTable(otherDbName, otherTable)) {
-        return false;
+        errors.add(datasetDescriptorPrefix + "." + DatasetDescriptorConfigKeys.TABLE_KEY + " is mismatched. User input: '" + otherTable + "' is in the blacklist"
+            + ".");

Review Comment:
   template these



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org