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 2020/08/06 10:14:09 UTC

[GitHub] [druid] clintropolis opened a new pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

clintropolis opened a new pull request #10248:
URL: https://github.com/apache/druid/pull/10248


   ### Description
   This PR fixes an issue when using `ExpressionVirtualColumn` expressions on realtime string columns which are sparsely populated and have not encountered explicit `null` values to ensure that they are encoded in the dictionary. 
   
   In the code there is an implicit assumption that if `ColumnCapabilities.isDictionaryEncoded` is true then `DimensionSelector.nameLookupPossibleInAdvance` is also true, and `isDictionaryEncoded` appears to be checked primarily in cases where this should also be true.
   
   Prior to this fix the added tests would explode with an error in the form:
   ```
   Selector of class[org.apache.druid.segment.StringDimensionIndexer$1IndexerDimensionSelector] does not have a dictionary, cannot use it.
   ```
   because [of this check](https://github.com/apache/druid/blob/master/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDimensionSelector.java#L52). 
   
   <hr>
   
   This PR has:
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `StringDimensionIndexer`
    * `ExpressionSelectors`
   


----------------------------------------------------------------
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 #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       Sorry, I don't understand the comment… are you saying that the laxer `isFalse()` version is the most correct one? Or are you saying it doesn't matter because unknowns aren't expected here?




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java
##########
@@ -2012,32 +2011,6 @@ public void testDictIdSeeker()
     Assert.assertEquals(-1, dictIdSeeker.seek(5));
   }
 
-  @Test(expected = IllegalArgumentException.class)
-  public void testCloser() throws Exception

Review comment:
       2 reasons, first, the mechanism by which it was causing the failure (artificially setting spatial index) doesn't work anymore since the capabilities are computed from the indexer rather than stored in a map. However, looking closer, the test didn't appear to be actually testing what it says (the closer cleaning up temporary files), because the paths it is checking for existence do not appear to be created anywhere in the code and only appear in this test, so removing it entirely didn't seem like a loss.




----------------------------------------------------------------
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 #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       Negative maybe makes me do a double-take when reading. Is this equivalent to `capabilities.isDictionaryEncoded.isFalse()`?
   
   Also, that has the effect of making this check lax (it will pass if the dictionary-encodedness is unknown). That doesn't seem correct. Shouldn't it be strict, if it relies on dictionary-encodedness? Or if it doesn't rely on it, then the check should be removed.

##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java
##########
@@ -236,6 +237,7 @@ DimensionSelector makeDimensionSelector(
       IncrementalIndex.DimensionDesc desc
   );
 
+  ColumnCapabilitiesImpl getColumnCapabilities();

Review comment:
       ColumnCapabilities is a more typical return value (the Impl is an impl but we should generally return an interface).

##########
File path: processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java
##########
@@ -400,6 +404,17 @@ public int getCardinality()
     return dimLookup.size();
   }
 
+  /**
+   * returns true if all values are encoded in {@link #dimLookup}
+   */
+  public boolean dictionaryEncodesAllValues()

Review comment:
       Can this be private? Looks like it isn't used outside of this class.

##########
File path: processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java
##########
@@ -456,6 +471,30 @@ public int getUnsortedEncodedKeyComponentHashCode(int[] key)
     return Arrays.hashCode(key);
   }
 
+  @Override
+  public ColumnCapabilitiesImpl getColumnCapabilities()
+  {
+    ColumnCapabilitiesImpl capabilites = new ColumnCapabilitiesImpl().setType(ValueType.STRING)
+                                                                     .setHasBitmapIndexes(hasBitmapIndexes)
+                                                                     .setHasSpatialIndexes(hasSpatialIndexes)
+                                                                     .setDictionaryValuesUnique(true)
+                                                                     .setDictionaryValuesSorted(false);
+
+    // strings are only single valued, until they are not...

Review comment:
       Please add some punctuation to this multiline comment 🙂
   
   I think I parsed it correctly, but it wasn't effortless.

##########
File path: processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java
##########
@@ -105,4 +104,46 @@ public String toString()
       return StringUtils.toLowerCase(super.toString());
     }
   }
+
+  interface CoercionLogic
+  {
+    boolean dictionaryEncoded();
+    boolean dictionaryValuesSorted();
+    boolean dictionaryValuesUnique();
+    boolean multipleValues();
+  }
+
+  class AllCoercionLogic implements CoercionLogic

Review comment:
       It looks like this is only used in the tests. If so, it should be moved there.

##########
File path: processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java
##########
@@ -105,4 +104,46 @@ public String toString()
       return StringUtils.toLowerCase(super.toString());
     }
   }
+
+  interface CoercionLogic

Review comment:
       Please add some javadocs. The purpose of this interface isn't immediately obvious.

##########
File path: processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java
##########
@@ -2012,32 +2011,6 @@ public void testDictIdSeeker()
     Assert.assertEquals(-1, dictIdSeeker.seek(5));
   }
 
-  @Test(expected = IllegalArgumentException.class)
-  public void testCloser() throws Exception

Review comment:
       Why does this test need to be deleted?

##########
File path: processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
##########
@@ -231,6 +252,80 @@ public void testCoerceExprToValue()
     );
   }
 
+  @Test
+  public void testIncrementIndexStringSelector() throws IndexSizeExceededException

Review comment:
       Is this the regression test? If so could you add a comment that this is a regression test for whichever the issue number is?




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java
##########
@@ -456,6 +471,30 @@ public int getUnsortedEncodedKeyComponentHashCode(int[] key)
     return Arrays.hashCode(key);
   }
 
+  @Override
+  public ColumnCapabilitiesImpl getColumnCapabilities()
+  {
+    ColumnCapabilitiesImpl capabilites = new ColumnCapabilitiesImpl().setType(ValueType.STRING)
+                                                                     .setHasBitmapIndexes(hasBitmapIndexes)
+                                                                     .setHasSpatialIndexes(hasSpatialIndexes)
+                                                                     .setDictionaryValuesUnique(true)
+                                                                     .setDictionaryValuesSorted(false);
+
+    // strings are only single valued, until they are not...

Review comment:
       reworked these comments, hopefully it is a bit clearer




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       There wasn't actually an `isFalse`, but I added it, and changed a few places to use it to read cleaner. The relaxed check isn't actually necessary here either due to the `CoerceLogic` changes, but I have left is as `isFalse()` instead instead of the slightly more strict `! ... isTrue()`.




----------------------------------------------------------------
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 #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       > This is because unknown would only happen from string dimension indexers of the incremental index which ultimately always produce string columns with a dictionary, and these handlers this method are making are used for things like index merging, etc, and so could safely treat unknown from the dim indexer as true because it wants the desired state rather than the current state.
   
   Thanks for explaining it. This sounds brittle for the following reason:
   
   1. At the call site here, the check really does mean to require a dictionary.
   2. So logically it should treat unknowns as false (to be safe — if it's not sure if a dictionary exists, and it requires one, it should fail the check).
   3. But it sounds like there are cases where an unknown will appear, and we "know" that it is really meant to be a true. So we can treat unknowns as OK.
   
   But what if a new case appears in a future patch that isn't (3)? Then this check is too lax. And because the new case is likely to come from a new call site of `getHandlerFromCapabilities`, whoever adds it will probably not realize they need to update the check here.
   
   So instead, would it work to keep the check strict, and resolving unknowns to trues at the current call sites? Then, if a new caller is added, it'd have to deal with the ambiguity.




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
##########
@@ -231,6 +252,80 @@ public void testCoerceExprToValue()
     );
   }
 
+  @Test
+  public void testIncrementIndexStringSelector() throws IndexSizeExceededException

Review comment:
       added a comment, (there isn't an open issue as far as I could tell)




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java
##########
@@ -105,4 +104,46 @@ public String toString()
       return StringUtils.toLowerCase(super.toString());
     }
   }
+
+  interface CoercionLogic

Review comment:
       Added some javadocs, also tried my hand at documenting `ColumnCapabilities` while i was there

##########
File path: processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java
##########
@@ -105,4 +104,46 @@ public String toString()
       return StringUtils.toLowerCase(super.toString());
     }
   }
+
+  interface CoercionLogic
+  {
+    boolean dictionaryEncoded();
+    boolean dictionaryValuesSorted();
+    boolean dictionaryValuesUnique();
+    boolean multipleValues();
+  }
+
+  class AllCoercionLogic implements CoercionLogic

Review comment:
       removed




----------------------------------------------------------------
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 #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       > which caught one I was mistakenly coercing out of band with `CoerceLogic`
   
   The system works!!




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
##########
@@ -231,6 +252,80 @@ public void testCoerceExprToValue()
     );
   }
 
+  @Test
+  public void testIncrementIndexStringSelector() throws IndexSizeExceededException

Review comment:
       added link to PR




----------------------------------------------------------------
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 #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       > This is because unknown would only happen from string dimension indexers of the incremental index which ultimately always produce string columns with a dictionary, and these handlers this method are making are used for things like index merging, etc, and so could safely treat unknown from the dim indexer as true because it wants the desired state rather than the current state.
   
   Thanks for explaining it. This sounds brittle for the following reason:
   
   1. At the call site here, the check really does mean to require a dictionary.
   2. So logically it should treat unknowns as false (to be safe — if it's not sure if a dictionary exists, and it requires one, then it should fail the check).
   3. But it sounds like there are cases where an unknown will appear, and we "know" that it is really meant to be a true. So we need to treat unknowns as OK instead of not OK.
   
   But what if a new case appears in a future patch that isn't (3)? Then this check is too lax. And because the new case is likely to come from a new call site of `getHandlerFromCapabilities`, whoever adds it will probably not realize they need to update the check here.
   
   So instead, would it work to keep the check strict, and resolving unknowns to trues at the current call sites? Then, if a new caller is added, it'd have to deal with the ambiguity.




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java
##########
@@ -236,6 +237,7 @@ DimensionSelector makeDimensionSelector(
       IncrementalIndex.DimensionDesc desc
   );
 
+  ColumnCapabilitiesImpl getColumnCapabilities();

Review comment:
       changed many places to use `ColumnCapabilities` instead of `ColumnCapabilitiesImpl`, since they are no longer set from multiple places




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       I hope this doesn't make this more confusing 😜 
   
   Unknowns are no longer expected to be here, but the unknowns which could previously appear here, prior to the coercion logic refactoring, should have been treated as true anyway due to the context of how this method is called. This is because unknown would only happen from string dimension indexers of the incremental index which ultimately always produce string columns with a dictionary, and these handlers this method are making are used for things like index merging, etc, and so could safely treat unknown from the dim indexer as true because it wants the desired state rather than the current state.
   
   Not going to do in this PR, but I am trying to plan out how to refactor `ColumnCapabilitiesImpl` into a builder that requires a coercion logic to build into a `ColumnCapabilities`, and changing `ColumnCapabilities` back to having only `boolean`, instead of the `Capable` enum. The idea being to limit the surface area of having to deal with unknown to where is absolutely necessary, because it makes things overly complex to have to deal with these tri-state booleans everywhere.




----------------------------------------------------------------
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 #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
##########
@@ -231,6 +252,80 @@ public void testCoerceExprToValue()
     );
   }
 
+  @Test
+  public void testIncrementIndexStringSelector() throws IndexSizeExceededException

Review comment:
       A link to this PR would suffice, IMO.




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       I changed the check back to the stricter check, and added more explicit calls to `snapshot` method around the callers of this method where necessary (which caught one I was mistakenly coercing out of band with `CoerceLogic`).




----------------------------------------------------------------
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] clintropolis commented on a change in pull request #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java
##########
@@ -400,6 +404,17 @@ public int getCardinality()
     return dimLookup.size();
   }
 
+  /**
+   * returns true if all values are encoded in {@link #dimLookup}
+   */
+  public boolean dictionaryEncodesAllValues()

Review comment:
       oops, yes, this is a relic from an earlier, uglier, version of this fix, it is private now




----------------------------------------------------------------
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 #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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



##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -73,16 +73,16 @@ private DimensionHandlerUtils()
   )
   {
     if (capabilities == null) {
-      return new StringDimensionHandler(dimensionName, multiValueHandling, true);
+      return new StringDimensionHandler(dimensionName, multiValueHandling, true, false);
     }
 
     multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
 
     if (capabilities.getType() == ValueType.STRING) {
-      if (!capabilities.isDictionaryEncoded()) {
+      if (!capabilities.isDictionaryEncoded().isMaybeTrue()) {

Review comment:
       > This is because unknown would only happen from string dimension indexers of the incremental index which ultimately always produce string columns with a dictionary, and these handlers this method are making are used for things like index merging, etc, and so could safely treat unknown from the dim indexer as true because it wants the desired state rather than the current state.
   
   Thanks for explaining it. This sounds brittle for the following reason:
   
   1. At the call site here, the check really does mean to require a dictionary.
   2. So logically it should treat unknowns as false (to be safe — if it's not sure if a dictionary exists, and it requires one, then it should fail the check).
   3. But it sounds like there are cases where an unknown will appear, and we "know" that it is really meant to be a true. So we can treat unknowns as OK.
   
   But what if a new case appears in a future patch that isn't (3)? Then this check is too lax. And because the new case is likely to come from a new call site of `getHandlerFromCapabilities`, whoever adds it will probably not realize they need to update the check here.
   
   So instead, would it work to keep the check strict, and resolving unknowns to trues at the current call sites? Then, if a new caller is added, it'd have to deal with the ambiguity.




----------------------------------------------------------------
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 #10248: fix bug with expressions on sparse string realtime columns without explicit null valued rows

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


   


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