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/11/05 22:03:10 UTC

[GitHub] [druid] gianm opened a new pull request #11884: RowBasedCursor: Add column-value-reuse optimization.

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


   Most of the logic is in RowBasedColumnSelectorFactory, although in this
   patch its only user is RowBasedCursor. This improves performance of
   features that use RowBasedSegment, like lookup and inline datasources.
   It's especially helpful for inline datasources that contain lengthy
   arrays, due to the fact that the transformed array can be reused.


-- 
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: commits-unsubscribe@druid.apache.org

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] clintropolis commented on a change in pull request #11884: RowBasedCursor: Add column-value-reuse optimization.

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



##########
File path: processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java
##########
@@ -340,9 +330,67 @@ public Class classOfObject()
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", supplier);
+          inspector.visit("row", rowSupplier);
           inspector.visit("extractionFn", extractionFn);
         }
+
+        private void updateCurrentValues()
+        {
+          if (rowIdSupplier == null || rowIdSupplier.getAsLong() != currentId) {
+            try {
+              final Object rawValue = dimFunction.apply(rowSupplier.get());
+
+              if (rawValue == null || rawValue instanceof String) {
+                final String s = NullHandling.emptyToNullIfNeeded((String) rawValue);
+
+                if (extractionFn == null) {
+                  dimensionValues = Collections.singletonList(s);
+                } else {
+                  dimensionValues = Collections.singletonList(extractionFn.apply(s));
+                }
+              } else if (rawValue instanceof List) {
+                // Consistent behavior with Rows.objectToStrings, but applies extractionFn too.
+                //noinspection rawtypes
+                final List<String> values = new ArrayList<>(((List) rawValue).size());
+
+                //noinspection rawtypes
+                for (final Object item : ((List) rawValue)) {
+                  // Behavior with null item is to convert it to string "null". This is not what most other areas of Druid
+                  // would do when treating a null as a string, but it's consistent with Rows.objectToStrings, which is
+                  // commonly used when retrieving strings from input-row-like objects.
+                  if (extractionFn == null) {
+                    values.add(NullHandling.emptyToNullIfNeeded(String.valueOf(item)));

Review comment:
       ~heh, i've never paid close attention to this, so that means `null` becomes `"null"` but `""` becomes `null`? but only if it is a list of strings, not a single string...~
   
   I don't think this needs done now, but I can't help but wonder if things that want this transformation to happen should like provide a mechanism to make it happen so that it isn't implicit functionality of the row selector stuff
   
   edit: my brain doesn't work right




-- 
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: commits-unsubscribe@druid.apache.org

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] clintropolis commented on a change in pull request #11884: RowBasedCursor: Add column-value-reuse optimization.

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



##########
File path: processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java
##########
@@ -340,9 +330,67 @@ public Class classOfObject()
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", supplier);
+          inspector.visit("row", rowSupplier);
           inspector.visit("extractionFn", extractionFn);
         }
+
+        private void updateCurrentValues()
+        {
+          if (rowIdSupplier == null || rowIdSupplier.getAsLong() != currentId) {
+            try {
+              final Object rawValue = dimFunction.apply(rowSupplier.get());
+
+              if (rawValue == null || rawValue instanceof String) {
+                final String s = NullHandling.emptyToNullIfNeeded((String) rawValue);
+
+                if (extractionFn == null) {
+                  dimensionValues = Collections.singletonList(s);
+                } else {
+                  dimensionValues = Collections.singletonList(extractionFn.apply(s));
+                }
+              } else if (rawValue instanceof List) {
+                // Consistent behavior with Rows.objectToStrings, but applies extractionFn too.
+                //noinspection rawtypes
+                final List<String> values = new ArrayList<>(((List) rawValue).size());
+
+                //noinspection rawtypes
+                for (final Object item : ((List) rawValue)) {
+                  // Behavior with null item is to convert it to string "null". This is not what most other areas of Druid
+                  // would do when treating a null as a string, but it's consistent with Rows.objectToStrings, which is
+                  // commonly used when retrieving strings from input-row-like objects.
+                  if (extractionFn == null) {
+                    values.add(NullHandling.emptyToNullIfNeeded(String.valueOf(item)));

Review comment:
       heh, i've never paid close attention to this, so that means `null` becomes `"null"` but `""` becomes `null`? but only if it is a list of strings, not a single string...
   
   I don't think this needs done now, but I can't help but wonder if things that want this transformation to happen should like provide a mechanism to make it happen so that it isn't implicit functionality of the row selector stuff




-- 
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: commits-unsubscribe@druid.apache.org

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 #11884: RowBasedCursor: Add column-value-reuse optimization.

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



##########
File path: processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java
##########
@@ -340,9 +330,67 @@ public Class classOfObject()
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", supplier);
+          inspector.visit("row", rowSupplier);
           inspector.visit("extractionFn", extractionFn);
         }
+
+        private void updateCurrentValues()
+        {
+          if (rowIdSupplier == null || rowIdSupplier.getAsLong() != currentId) {
+            try {
+              final Object rawValue = dimFunction.apply(rowSupplier.get());
+
+              if (rawValue == null || rawValue instanceof String) {
+                final String s = NullHandling.emptyToNullIfNeeded((String) rawValue);
+
+                if (extractionFn == null) {
+                  dimensionValues = Collections.singletonList(s);
+                } else {
+                  dimensionValues = Collections.singletonList(extractionFn.apply(s));
+                }
+              } else if (rawValue instanceof List) {
+                // Consistent behavior with Rows.objectToStrings, but applies extractionFn too.
+                //noinspection rawtypes
+                final List<String> values = new ArrayList<>(((List) rawValue).size());
+
+                //noinspection rawtypes
+                for (final Object item : ((List) rawValue)) {
+                  // Behavior with null item is to convert it to string "null". This is not what most other areas of Druid
+                  // would do when treating a null as a string, but it's consistent with Rows.objectToStrings, which is
+                  // commonly used when retrieving strings from input-row-like objects.
+                  if (extractionFn == null) {
+                    values.add(NullHandling.emptyToNullIfNeeded(String.valueOf(item)));

Review comment:
       My goal here was to retain the old behavior of RowBasedColumnSelectorFactory (weird as it is), and just add the caching optimization. So I'd like to leave the behavior toggles to a different patch, if you don't mind. I haven't thought about it enough to have a good proposal right now.
   
   Although, about retaining the old behavior: after looking into it more, I realized this patch isn't doing it. I need to move the NullHandling.emptyToNullIfNeeded from here to lookupName to retain the current behavior. So, I'll do that for this patch.




-- 
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: commits-unsubscribe@druid.apache.org

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] clintropolis commented on a change in pull request #11884: RowBasedCursor: Add column-value-reuse optimization.

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



##########
File path: processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java
##########
@@ -340,9 +330,67 @@ public Class classOfObject()
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", supplier);
+          inspector.visit("row", rowSupplier);
           inspector.visit("extractionFn", extractionFn);
         }
+
+        private void updateCurrentValues()
+        {
+          if (rowIdSupplier == null || rowIdSupplier.getAsLong() != currentId) {
+            try {
+              final Object rawValue = dimFunction.apply(rowSupplier.get());
+
+              if (rawValue == null || rawValue instanceof String) {
+                final String s = NullHandling.emptyToNullIfNeeded((String) rawValue);
+
+                if (extractionFn == null) {
+                  dimensionValues = Collections.singletonList(s);
+                } else {
+                  dimensionValues = Collections.singletonList(extractionFn.apply(s));
+                }
+              } else if (rawValue instanceof List) {
+                // Consistent behavior with Rows.objectToStrings, but applies extractionFn too.
+                //noinspection rawtypes
+                final List<String> values = new ArrayList<>(((List) rawValue).size());
+
+                //noinspection rawtypes
+                for (final Object item : ((List) rawValue)) {
+                  // Behavior with null item is to convert it to string "null". This is not what most other areas of Druid
+                  // would do when treating a null as a string, but it's consistent with Rows.objectToStrings, which is
+                  // commonly used when retrieving strings from input-row-like objects.
+                  if (extractionFn == null) {
+                    values.add(NullHandling.emptyToNullIfNeeded(String.valueOf(item)));

Review comment:
       heh, i've never paid close attention to this, so that means `null` becomes `"null"` but `""` becomes `null`? but only if it is a list of strings, not a single string... Or should these be reversed?
   
   ```values.add(String.valueOf(NullHandling.emptyToNullIfNeeded(item)));```
   
   I don't think this needs done now, but I can't help but wonder if things that want this transformation to happen should like provide a mechanism to make it happen so that it isn't implicit functionality of the row selector stuff




-- 
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: commits-unsubscribe@druid.apache.org

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 #11884: RowBasedCursor: Add column-value-reuse optimization.

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



##########
File path: processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java
##########
@@ -47,59 +51,71 @@
  */
 public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
 {
-  private final Supplier<T> supplier;
+  private static final long NO_ID = -1;
+
+  private final Supplier<T> rowSupplier;
+
+  @Nullable
+  private final LongSupplier rowIdSupplier;
   private final RowAdapter<T> adapter;
-  private final RowSignature rowSignature;
+  private final ColumnInspector columnInspector;
   private final boolean throwParseExceptions;
 
-  private RowBasedColumnSelectorFactory(
-      final Supplier<T> supplier,
+  /**
+   * Package-private constructor for {@link RowBasedCursor}. Allows passing in a rowIdSupplier, which enables
+   * column value reuse optimizations.
+   */
+  RowBasedColumnSelectorFactory(
+      final Supplier<T> rowSupplier,
+      @Nullable final LongSupplier rowIdSupplier,
       final RowAdapter<T> adapter,
-      final RowSignature rowSignature,
+      final ColumnInspector columnInspector,
       final boolean throwParseExceptions
   )
   {
-    this.supplier = supplier;
-    this.adapter = adapter;
-    this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature must be nonnull");
+    this.rowSupplier = Preconditions.checkNotNull(rowSupplier, "rowSupplier");
+    this.rowIdSupplier = rowIdSupplier;
+    this.adapter = Preconditions.checkNotNull(adapter, "adapter");
+    this.columnInspector = Preconditions.checkNotNull(columnInspector, "columnInspector must be nonnull");
     this.throwParseExceptions = throwParseExceptions;
   }
 
   /**
    * Create an instance based on any object, along with a {@link RowAdapter} for that object.
    *
    * @param adapter              adapter for these row objects
-   * @param supplier             supplier of row objects
-   * @param signature            will be used for reporting available columns and their capabilities. Note that the this
+   * @param rowSupplier          supplier of row objects
+   * @param columnInspector      will be used for reporting available columns and their capabilities. Note that this
    *                             factory will still allow creation of selectors on any named field in the rows, even if
-   *                             it doesn't appear in "rowSignature". (It only needs to be accessible via
+   *                             it doesn't appear in "columnInspector". (It only needs to be accessible via
    *                             {@link RowAdapter#columnFunction}.) As a result, you can achieve an untyped mode by
    *                             passing in {@link RowSignature#empty()}.
    * @param throwParseExceptions whether numeric selectors should throw parse exceptions or use a default/null value
    *                             when their inputs are not actually numeric
    */
   public static <RowType> RowBasedColumnSelectorFactory<RowType> create(
       final RowAdapter<RowType> adapter,
-      final Supplier<RowType> supplier,
-      final RowSignature signature,
+      final Supplier<RowType> rowSupplier,
+      final ColumnInspector columnInspector,
       final boolean throwParseExceptions
   )
   {
-    return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature, throwParseExceptions);
+    return new RowBasedColumnSelectorFactory<>(rowSupplier, null, adapter, columnInspector, throwParseExceptions);
   }
 
   @Nullable
   static ColumnCapabilities getColumnCapabilities(
-      final RowSignature rowSignature,
+      final ColumnInspector columnInspector,
       final String columnName
   )
   {
     if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) {
-      // TIME_COLUMN_NAME is handled specially; override the provided rowSignature.
+      // TIME_COLUMN_NAME is handled specially; override the provided inspector.
       return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG);
     } else {
-      final ColumnType valueType = rowSignature.getColumnType(columnName).orElse(null);
-
+      final Optional<ColumnCapabilities> inspectedCapabilities =
+          Optional.ofNullable(columnInspector.getColumnCapabilities(columnName));
+      final ColumnType valueType = inspectedCapabilities.map(ColumnCapabilities::toColumnType).orElse(null);

Review comment:
       Sure, I changed `ColumnCapabilitiesImpl.setType` and also `ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities` and `ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities`.




-- 
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: commits-unsubscribe@druid.apache.org

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] clintropolis commented on a change in pull request #11884: RowBasedCursor: Add column-value-reuse optimization.

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



##########
File path: processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java
##########
@@ -340,9 +330,67 @@ public Class classOfObject()
         @Override
         public void inspectRuntimeShape(RuntimeShapeInspector inspector)
         {
-          inspector.visit("row", supplier);
+          inspector.visit("row", rowSupplier);
           inspector.visit("extractionFn", extractionFn);
         }
+
+        private void updateCurrentValues()
+        {
+          if (rowIdSupplier == null || rowIdSupplier.getAsLong() != currentId) {
+            try {
+              final Object rawValue = dimFunction.apply(rowSupplier.get());
+
+              if (rawValue == null || rawValue instanceof String) {
+                final String s = NullHandling.emptyToNullIfNeeded((String) rawValue);
+
+                if (extractionFn == null) {
+                  dimensionValues = Collections.singletonList(s);
+                } else {
+                  dimensionValues = Collections.singletonList(extractionFn.apply(s));
+                }
+              } else if (rawValue instanceof List) {
+                // Consistent behavior with Rows.objectToStrings, but applies extractionFn too.
+                //noinspection rawtypes
+                final List<String> values = new ArrayList<>(((List) rawValue).size());
+
+                //noinspection rawtypes
+                for (final Object item : ((List) rawValue)) {
+                  // Behavior with null item is to convert it to string "null". This is not what most other areas of Druid
+                  // would do when treating a null as a string, but it's consistent with Rows.objectToStrings, which is
+                  // commonly used when retrieving strings from input-row-like objects.
+                  if (extractionFn == null) {
+                    values.add(NullHandling.emptyToNullIfNeeded(String.valueOf(item)));

Review comment:
       heh, i've never paid close attention to this, so that means `null` becomes `"null"` but `""` becomes `null`? but only if it is a list of strings, not a single string...
   
   I don't think this needs done now, but I can't help but wonder if things that want this transformation to happen should like provide a mechanism to make it happen so that it isn't implicit functionality of the row selector stuff

##########
File path: processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java
##########
@@ -47,59 +51,71 @@
  */
 public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
 {
-  private final Supplier<T> supplier;
+  private static final long NO_ID = -1;
+
+  private final Supplier<T> rowSupplier;
+
+  @Nullable
+  private final LongSupplier rowIdSupplier;
   private final RowAdapter<T> adapter;
-  private final RowSignature rowSignature;
+  private final ColumnInspector columnInspector;
   private final boolean throwParseExceptions;
 
-  private RowBasedColumnSelectorFactory(
-      final Supplier<T> supplier,
+  /**
+   * Package-private constructor for {@link RowBasedCursor}. Allows passing in a rowIdSupplier, which enables
+   * column value reuse optimizations.
+   */
+  RowBasedColumnSelectorFactory(
+      final Supplier<T> rowSupplier,
+      @Nullable final LongSupplier rowIdSupplier,
       final RowAdapter<T> adapter,
-      final RowSignature rowSignature,
+      final ColumnInspector columnInspector,
       final boolean throwParseExceptions
   )
   {
-    this.supplier = supplier;
-    this.adapter = adapter;
-    this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature must be nonnull");
+    this.rowSupplier = Preconditions.checkNotNull(rowSupplier, "rowSupplier");
+    this.rowIdSupplier = rowIdSupplier;
+    this.adapter = Preconditions.checkNotNull(adapter, "adapter");
+    this.columnInspector = Preconditions.checkNotNull(columnInspector, "columnInspector must be nonnull");
     this.throwParseExceptions = throwParseExceptions;
   }
 
   /**
    * Create an instance based on any object, along with a {@link RowAdapter} for that object.
    *
    * @param adapter              adapter for these row objects
-   * @param supplier             supplier of row objects
-   * @param signature            will be used for reporting available columns and their capabilities. Note that the this
+   * @param rowSupplier          supplier of row objects
+   * @param columnInspector      will be used for reporting available columns and their capabilities. Note that this
    *                             factory will still allow creation of selectors on any named field in the rows, even if
-   *                             it doesn't appear in "rowSignature". (It only needs to be accessible via
+   *                             it doesn't appear in "columnInspector". (It only needs to be accessible via
    *                             {@link RowAdapter#columnFunction}.) As a result, you can achieve an untyped mode by
    *                             passing in {@link RowSignature#empty()}.
    * @param throwParseExceptions whether numeric selectors should throw parse exceptions or use a default/null value
    *                             when their inputs are not actually numeric
    */
   public static <RowType> RowBasedColumnSelectorFactory<RowType> create(
       final RowAdapter<RowType> adapter,
-      final Supplier<RowType> supplier,
-      final RowSignature signature,
+      final Supplier<RowType> rowSupplier,
+      final ColumnInspector columnInspector,
       final boolean throwParseExceptions
   )
   {
-    return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature, throwParseExceptions);
+    return new RowBasedColumnSelectorFactory<>(rowSupplier, null, adapter, columnInspector, throwParseExceptions);
   }
 
   @Nullable
   static ColumnCapabilities getColumnCapabilities(
-      final RowSignature rowSignature,
+      final ColumnInspector columnInspector,
       final String columnName
   )
   {
     if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) {
-      // TIME_COLUMN_NAME is handled specially; override the provided rowSignature.
+      // TIME_COLUMN_NAME is handled specially; override the provided inspector.
       return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG);
     } else {
-      final ColumnType valueType = rowSignature.getColumnType(columnName).orElse(null);
-
+      final Optional<ColumnCapabilities> inspectedCapabilities =
+          Optional.ofNullable(columnInspector.getColumnCapabilities(columnName));
+      final ColumnType valueType = inspectedCapabilities.map(ColumnCapabilities::toColumnType).orElse(null);

Review comment:
       nit: I should have probably made `ColumnCapabilitiesImpl.setType` accept `TypeSignature<ValueType>` instead of `ColumnType` so that you could just use the capabilities from the inspector directly, want to make this change? 




-- 
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: commits-unsubscribe@druid.apache.org

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 #11884: RowBasedCursor: Add column-value-reuse optimization.

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


   


-- 
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: commits-unsubscribe@druid.apache.org

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