You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/04/08 17:54:02 UTC

[GitHub] [druid] gianm opened a new pull request #11082: DataSchema: Improve duplicate-column error message.

gianm opened a new pull request #11082:
URL: https://github.com/apache/druid/pull/11082


   Now, when duplicate columns are specified, the error message will include
   information about where those duplicate columns were seen. Also, if there
   are multiple duplicate columns, all will be listed in the error message
   instead of just the first one encountered.


-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a change in pull request #11082: DataSchema: Improve duplicate-column error message.

Posted by GitBox <gi...@apache.org>.
kfaraz commented on a change in pull request #11082:
URL: https://github.com/apache/druid/pull/11082#discussion_r610826766



##########
File path: server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java
##########
@@ -192,6 +176,89 @@ private static DimensionsSpec computeDimensionsSpec(
     return dimensionsSpec.withDimensionExclusions(additionalDimensionExclusions);
   }
 
+  private static Set<String> computeInputFieldNames(
+      final TimestampSpec timestampSpec,
+      final DimensionsSpec dimensionsSpec,
+      final AggregatorFactory[] aggregators
+  )
+  {
+    final Set<String> fields = new HashSet<>();
+
+    fields.add(timestampSpec.getTimestampColumn());
+    fields.addAll(dimensionsSpec.getDimensionNames());
+    Arrays.stream(aggregators)
+          .flatMap(aggregator -> aggregator.requiredFields().stream())
+          .forEach(fields::add);
+
+    return fields;
+  }
+
+  /**
+   * Computes the set of field names that are specified by the provided dimensions and aggregator lists.
+   *
+   * If either list is null, it is ignored.
+   *
+   * @throws IllegalArgumentException if there are duplicate field names, or if any dimension or aggregator
+   *                                  has a null name
+   */
+  private static Set<String> computeAndValidateOutputFieldNames(
+      @Nullable final DimensionsSpec dimensionsSpec,
+      @Nullable final AggregatorFactory[] aggregators
+  )
+  {
+    // Field name -> where it was seen
+    final Map<String, Multiset<String>> fields = new TreeMap<>();
+
+    fields.computeIfAbsent(ColumnHolder.TIME_COLUMN_NAME, k -> TreeMultiset.create()).add(
+        StringUtils.format(
+            "primary timestamp (%s cannot appear as a dimension or metric)",
+            ColumnHolder.TIME_COLUMN_NAME
+        )
+    );
+
+    if (dimensionsSpec != null) {
+      for (int i = 0; i < dimensionsSpec.getDimensions().size(); i++) {
+        final String field = dimensionsSpec.getDimensions().get(i).getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered dimension with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("dimensions list");
+      }
+    }
+
+    if (aggregators != null) {
+      for (int i = 0; i < aggregators.length; i++) {
+        final String field = aggregators[i].getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered metric with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("metricsSpec list");
+      }
+    }
+
+    final List<String> errors = new ArrayList<>();
+
+    for (Map.Entry<String, Multiset<String>> entry : fields.entrySet()) {
+      if (entry.getValue().entrySet().stream().mapToInt(Multiset.Entry::getCount).sum() > 1) {
+        errors.add(
+            StringUtils.format(
+                "[%s] seen in %s",

Review comment:
       Since we are using a Multiset anyway, we would have the count against each element in the set. So I guess we could show some message like:
   ```
   Cannot specify a column more than once: [metric1] seen in metricsSpec list (2 occurrences)
   ```




-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #11082: DataSchema: Improve duplicate-column error message.

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #11082:
URL: https://github.com/apache/druid/pull/11082#discussion_r610769060



##########
File path: server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java
##########
@@ -192,6 +176,89 @@ private static DimensionsSpec computeDimensionsSpec(
     return dimensionsSpec.withDimensionExclusions(additionalDimensionExclusions);
   }
 
+  private static Set<String> computeInputFieldNames(
+      final TimestampSpec timestampSpec,
+      final DimensionsSpec dimensionsSpec,
+      final AggregatorFactory[] aggregators
+  )
+  {
+    final Set<String> fields = new HashSet<>();
+
+    fields.add(timestampSpec.getTimestampColumn());
+    fields.addAll(dimensionsSpec.getDimensionNames());
+    Arrays.stream(aggregators)
+          .flatMap(aggregator -> aggregator.requiredFields().stream())
+          .forEach(fields::add);
+
+    return fields;
+  }
+
+  /**
+   * Computes the set of field names that are specified by the provided dimensions and aggregator lists.
+   *
+   * If either list is null, it is ignored.
+   *
+   * @throws IllegalArgumentException if there are duplicate field names, or if any dimension or aggregator
+   *                                  has a null name
+   */
+  private static Set<String> computeAndValidateOutputFieldNames(
+      @Nullable final DimensionsSpec dimensionsSpec,
+      @Nullable final AggregatorFactory[] aggregators
+  )
+  {
+    // Field name -> where it was seen
+    final Map<String, Multiset<String>> fields = new TreeMap<>();
+
+    fields.computeIfAbsent(ColumnHolder.TIME_COLUMN_NAME, k -> TreeMultiset.create()).add(
+        StringUtils.format(
+            "primary timestamp (%s cannot appear as a dimension or metric)",
+            ColumnHolder.TIME_COLUMN_NAME
+        )
+    );
+
+    if (dimensionsSpec != null) {
+      for (int i = 0; i < dimensionsSpec.getDimensions().size(); i++) {
+        final String field = dimensionsSpec.getDimensions().get(i).getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered dimension with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("dimensions list");
+      }
+    }
+
+    if (aggregators != null) {
+      for (int i = 0; i < aggregators.length; i++) {
+        final String field = aggregators[i].getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered metric with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("metricsSpec list");
+      }
+    }
+
+    final List<String> errors = new ArrayList<>();
+
+    for (Map.Entry<String, Multiset<String>> entry : fields.entrySet()) {
+      if (entry.getValue().entrySet().stream().mapToInt(Multiset.Entry::getCount).sum() > 1) {
+        errors.add(
+            StringUtils.format(
+                "[%s] seen in %s",

Review comment:
       It _could_ happen, but I was thinking it would still be understandable. The message would be like:
   
   ```
   Cannot specify a column more than once: [metric1] seen in metricsSpec list
   ```
   
   I was thinking that the user would be able to look at the metricsSpec and see that `metric1` is there more than once. But I'm happy to take suggestions on better error messages.




-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a change in pull request #11082: DataSchema: Improve duplicate-column error message.

Posted by GitBox <gi...@apache.org>.
kfaraz commented on a change in pull request #11082:
URL: https://github.com/apache/druid/pull/11082#discussion_r610765870



##########
File path: server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java
##########
@@ -192,6 +176,89 @@ private static DimensionsSpec computeDimensionsSpec(
     return dimensionsSpec.withDimensionExclusions(additionalDimensionExclusions);
   }
 
+  private static Set<String> computeInputFieldNames(
+      final TimestampSpec timestampSpec,
+      final DimensionsSpec dimensionsSpec,
+      final AggregatorFactory[] aggregators
+  )
+  {
+    final Set<String> fields = new HashSet<>();
+
+    fields.add(timestampSpec.getTimestampColumn());
+    fields.addAll(dimensionsSpec.getDimensionNames());
+    Arrays.stream(aggregators)
+          .flatMap(aggregator -> aggregator.requiredFields().stream())
+          .forEach(fields::add);
+
+    return fields;
+  }
+
+  /**
+   * Computes the set of field names that are specified by the provided dimensions and aggregator lists.
+   *
+   * If either list is null, it is ignored.
+   *
+   * @throws IllegalArgumentException if there are duplicate field names, or if any dimension or aggregator
+   *                                  has a null name
+   */
+  private static Set<String> computeAndValidateOutputFieldNames(
+      @Nullable final DimensionsSpec dimensionsSpec,
+      @Nullable final AggregatorFactory[] aggregators
+  )
+  {
+    // Field name -> where it was seen
+    final Map<String, Multiset<String>> fields = new TreeMap<>();
+
+    fields.computeIfAbsent(ColumnHolder.TIME_COLUMN_NAME, k -> TreeMultiset.create()).add(
+        StringUtils.format(
+            "primary timestamp (%s cannot appear as a dimension or metric)",
+            ColumnHolder.TIME_COLUMN_NAME
+        )
+    );
+
+    if (dimensionsSpec != null) {
+      for (int i = 0; i < dimensionsSpec.getDimensions().size(); i++) {
+        final String field = dimensionsSpec.getDimensions().get(i).getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered dimension with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("dimensions list");
+      }
+    }
+
+    if (aggregators != null) {
+      for (int i = 0; i < aggregators.length; i++) {
+        final String field = aggregators[i].getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered metric with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("metricsSpec list");
+      }
+    }
+
+    final List<String> errors = new ArrayList<>();
+
+    for (Map.Entry<String, Multiset<String>> entry : fields.entrySet()) {
+      if (entry.getValue().entrySet().stream().mapToInt(Multiset.Entry::getCount).sum() > 1) {
+        errors.add(
+            StringUtils.format(
+                "[%s] seen in %s",

Review comment:
       This message might not be sufficient if a field appears more than once either in the `dimensionsSpec` or in the `metricsSpec` but I guess that will never happen?




-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a change in pull request #11082: DataSchema: Improve duplicate-column error message.

Posted by GitBox <gi...@apache.org>.
kfaraz commented on a change in pull request #11082:
URL: https://github.com/apache/druid/pull/11082#discussion_r610826766



##########
File path: server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java
##########
@@ -192,6 +176,89 @@ private static DimensionsSpec computeDimensionsSpec(
     return dimensionsSpec.withDimensionExclusions(additionalDimensionExclusions);
   }
 
+  private static Set<String> computeInputFieldNames(
+      final TimestampSpec timestampSpec,
+      final DimensionsSpec dimensionsSpec,
+      final AggregatorFactory[] aggregators
+  )
+  {
+    final Set<String> fields = new HashSet<>();
+
+    fields.add(timestampSpec.getTimestampColumn());
+    fields.addAll(dimensionsSpec.getDimensionNames());
+    Arrays.stream(aggregators)
+          .flatMap(aggregator -> aggregator.requiredFields().stream())
+          .forEach(fields::add);
+
+    return fields;
+  }
+
+  /**
+   * Computes the set of field names that are specified by the provided dimensions and aggregator lists.
+   *
+   * If either list is null, it is ignored.
+   *
+   * @throws IllegalArgumentException if there are duplicate field names, or if any dimension or aggregator
+   *                                  has a null name
+   */
+  private static Set<String> computeAndValidateOutputFieldNames(
+      @Nullable final DimensionsSpec dimensionsSpec,
+      @Nullable final AggregatorFactory[] aggregators
+  )
+  {
+    // Field name -> where it was seen
+    final Map<String, Multiset<String>> fields = new TreeMap<>();
+
+    fields.computeIfAbsent(ColumnHolder.TIME_COLUMN_NAME, k -> TreeMultiset.create()).add(
+        StringUtils.format(
+            "primary timestamp (%s cannot appear as a dimension or metric)",
+            ColumnHolder.TIME_COLUMN_NAME
+        )
+    );
+
+    if (dimensionsSpec != null) {
+      for (int i = 0; i < dimensionsSpec.getDimensions().size(); i++) {
+        final String field = dimensionsSpec.getDimensions().get(i).getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered dimension with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("dimensions list");
+      }
+    }
+
+    if (aggregators != null) {
+      for (int i = 0; i < aggregators.length; i++) {
+        final String field = aggregators[i].getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered metric with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("metricsSpec list");
+      }
+    }
+
+    final List<String> errors = new ArrayList<>();
+
+    for (Map.Entry<String, Multiset<String>> entry : fields.entrySet()) {
+      if (entry.getValue().entrySet().stream().mapToInt(Multiset.Entry::getCount).sum() > 1) {
+        errors.add(
+            StringUtils.format(
+                "[%s] seen in %s",

Review comment:
       Since we are using a Multiset anyway, I guess we could show some message like:
   ```
   Cannot specify a column more than once: [metric1] seen in metricsSpec list (2 occurrences)
   ```




-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #11082: DataSchema: Improve duplicate-column error message.

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #11082:
URL: https://github.com/apache/druid/pull/11082#discussion_r611693983



##########
File path: server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java
##########
@@ -192,6 +176,89 @@ private static DimensionsSpec computeDimensionsSpec(
     return dimensionsSpec.withDimensionExclusions(additionalDimensionExclusions);
   }
 
+  private static Set<String> computeInputFieldNames(
+      final TimestampSpec timestampSpec,
+      final DimensionsSpec dimensionsSpec,
+      final AggregatorFactory[] aggregators
+  )
+  {
+    final Set<String> fields = new HashSet<>();
+
+    fields.add(timestampSpec.getTimestampColumn());
+    fields.addAll(dimensionsSpec.getDimensionNames());
+    Arrays.stream(aggregators)
+          .flatMap(aggregator -> aggregator.requiredFields().stream())
+          .forEach(fields::add);
+
+    return fields;
+  }
+
+  /**
+   * Computes the set of field names that are specified by the provided dimensions and aggregator lists.
+   *
+   * If either list is null, it is ignored.
+   *
+   * @throws IllegalArgumentException if there are duplicate field names, or if any dimension or aggregator
+   *                                  has a null name
+   */
+  private static Set<String> computeAndValidateOutputFieldNames(
+      @Nullable final DimensionsSpec dimensionsSpec,
+      @Nullable final AggregatorFactory[] aggregators
+  )
+  {
+    // Field name -> where it was seen
+    final Map<String, Multiset<String>> fields = new TreeMap<>();
+
+    fields.computeIfAbsent(ColumnHolder.TIME_COLUMN_NAME, k -> TreeMultiset.create()).add(
+        StringUtils.format(
+            "primary timestamp (%s cannot appear as a dimension or metric)",
+            ColumnHolder.TIME_COLUMN_NAME
+        )
+    );
+
+    if (dimensionsSpec != null) {
+      for (int i = 0; i < dimensionsSpec.getDimensions().size(); i++) {
+        final String field = dimensionsSpec.getDimensions().get(i).getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered dimension with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("dimensions list");
+      }
+    }
+
+    if (aggregators != null) {
+      for (int i = 0; i < aggregators.length; i++) {
+        final String field = aggregators[i].getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered metric with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("metricsSpec list");
+      }
+    }
+
+    final List<String> errors = new ArrayList<>();
+
+    for (Map.Entry<String, Multiset<String>> entry : fields.entrySet()) {
+      if (entry.getValue().entrySet().stream().mapToInt(Multiset.Entry::getCount).sum() > 1) {
+        errors.add(
+            StringUtils.format(
+                "[%s] seen in %s",

Review comment:
       @kfaraz, good suggestion, I have implemented it.




-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a change in pull request #11082: DataSchema: Improve duplicate-column error message.

Posted by GitBox <gi...@apache.org>.
kfaraz commented on a change in pull request #11082:
URL: https://github.com/apache/druid/pull/11082#discussion_r610826766



##########
File path: server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java
##########
@@ -192,6 +176,89 @@ private static DimensionsSpec computeDimensionsSpec(
     return dimensionsSpec.withDimensionExclusions(additionalDimensionExclusions);
   }
 
+  private static Set<String> computeInputFieldNames(
+      final TimestampSpec timestampSpec,
+      final DimensionsSpec dimensionsSpec,
+      final AggregatorFactory[] aggregators
+  )
+  {
+    final Set<String> fields = new HashSet<>();
+
+    fields.add(timestampSpec.getTimestampColumn());
+    fields.addAll(dimensionsSpec.getDimensionNames());
+    Arrays.stream(aggregators)
+          .flatMap(aggregator -> aggregator.requiredFields().stream())
+          .forEach(fields::add);
+
+    return fields;
+  }
+
+  /**
+   * Computes the set of field names that are specified by the provided dimensions and aggregator lists.
+   *
+   * If either list is null, it is ignored.
+   *
+   * @throws IllegalArgumentException if there are duplicate field names, or if any dimension or aggregator
+   *                                  has a null name
+   */
+  private static Set<String> computeAndValidateOutputFieldNames(
+      @Nullable final DimensionsSpec dimensionsSpec,
+      @Nullable final AggregatorFactory[] aggregators
+  )
+  {
+    // Field name -> where it was seen
+    final Map<String, Multiset<String>> fields = new TreeMap<>();
+
+    fields.computeIfAbsent(ColumnHolder.TIME_COLUMN_NAME, k -> TreeMultiset.create()).add(
+        StringUtils.format(
+            "primary timestamp (%s cannot appear as a dimension or metric)",
+            ColumnHolder.TIME_COLUMN_NAME
+        )
+    );
+
+    if (dimensionsSpec != null) {
+      for (int i = 0; i < dimensionsSpec.getDimensions().size(); i++) {
+        final String field = dimensionsSpec.getDimensions().get(i).getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered dimension with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("dimensions list");
+      }
+    }
+
+    if (aggregators != null) {
+      for (int i = 0; i < aggregators.length; i++) {
+        final String field = aggregators[i].getName();
+        if (Strings.isNullOrEmpty(field)) {
+          throw new IAE("Encountered metric with null or empty name at position %d", i);
+        }
+
+        fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("metricsSpec list");
+      }
+    }
+
+    final List<String> errors = new ArrayList<>();
+
+    for (Map.Entry<String, Multiset<String>> entry : fields.entrySet()) {
+      if (entry.getValue().entrySet().stream().mapToInt(Multiset.Entry::getCount).sum() > 1) {
+        errors.add(
+            StringUtils.format(
+                "[%s] seen in %s",

Review comment:
       Since we are using a Multiset anyway, we would have the count against each element in the set. So I guess we could show some message like:
   ```
   Cannot specify a column more than once: [metric1] seen in metricsSpec list (2 occurrences), dimensions list (1 occurrence)
   ```
   
   We could even just use `toString()` of the Multiset as for a Multiset `{a, a, b}`, it gives a String `a x 2, b`




-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm merged pull request #11082: DataSchema: Improve duplicate-column error message.

Posted by GitBox <gi...@apache.org>.
gianm merged pull request #11082:
URL: https://github.com/apache/druid/pull/11082


   


-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org