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/06/09 21:21:29 UTC

[GitHub] [druid] maytasm commented on a change in pull request #9994: remove incorrect and unnecessary overrides from BooleanVectorValueMatcher

maytasm commented on a change in pull request #9994:
URL: https://github.com/apache/druid/pull/9994#discussion_r437729822



##########
File path: processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java
##########
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.filter.vector;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.segment.IdLookup;
+import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.VectorValueSelector;
+import org.apache.druid.testing.InitializedNullHandlingTest;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNullHandlingTest
+{
+  private static final int VECTOR_SIZE = 128;
+  private static final int CURRENT_SIZE = 24;
+  private VectorValueSelector vectorValueSelector;
+
+  @Before
+  public void setup()
+  {
+    vectorValueSelector = EasyMock.createMock(VectorValueSelector.class);
+    EasyMock.expect(vectorValueSelector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes();
+    EasyMock.expect(vectorValueSelector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes();
+    EasyMock.replay(vectorValueSelector);
+  }
+
+  @Test
+  public void testFloat()
+  {
+    VectorValueMatcherFactory matcherFactory =
+        VectorValueMatcherColumnProcessorFactory.instance().makeFloatProcessor(vectorValueSelector);
+
+    Assert.assertTrue(matcherFactory instanceof FloatVectorValueMatcher);
+
+    VectorValueMatcher matcher = matcherFactory.makeMatcher("2.0");
+    Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher);
+    Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize());
+    Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize());
+
+    // in default mode, matching null produces a boolean matcher
+    VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null);
+    if (NullHandling.replaceWithDefault()) {
+      Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher);
+    } else {
+      Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
+    }
+    Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize());
+    Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize());
+    EasyMock.verify(vectorValueSelector);
+  }
+
+  @Test
+  public void testDouble()
+  {
+    VectorValueMatcherFactory matcherFactory =
+        VectorValueMatcherColumnProcessorFactory.instance().makeDoubleProcessor(vectorValueSelector);
+
+    Assert.assertTrue(matcherFactory instanceof DoubleVectorValueMatcher);
+
+
+    VectorValueMatcher matcher = matcherFactory.makeMatcher("1.0");
+    Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher);
+    Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize());
+    Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize());
+
+    // in default mode, matching null produces a boolean matcher
+    VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null);
+    if (NullHandling.replaceWithDefault()) {
+      Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher);
+    } else {
+      Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
+    }
+    Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize());
+    Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize());
+    EasyMock.verify(vectorValueSelector);
+  }
+
+  @Test
+  public void testLong()
+  {
+    VectorValueMatcherFactory matcherFactory =
+        VectorValueMatcherColumnProcessorFactory.instance().makeLongProcessor(vectorValueSelector);
+
+    Assert.assertTrue(matcherFactory instanceof LongVectorValueMatcher);
+
+    VectorValueMatcher matcher = matcherFactory.makeMatcher("1");
+    Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher);
+    Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize());
+    Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize());
+
+    // in default mode, matching null produces a boolean matcher
+    VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null);
+    if (NullHandling.replaceWithDefault()) {
+      Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher);
+    } else {
+      Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
+    }
+    Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize());
+    Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize());
+    EasyMock.verify(vectorValueSelector);
+  }
+
+  @Test
+  public void testSingleValueString()
+  {
+    IdLookup lookup = EasyMock.createMock(IdLookup.class);
+    SingleValueDimensionVectorSelector selector =
+        EasyMock.createMock(SingleValueDimensionVectorSelector.class);
+    EasyMock.expect(selector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes();
+    EasyMock.expect(selector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes();
+    EasyMock.expect(selector.getValueCardinality()).andReturn(1024).anyTimes();
+    EasyMock.expect(selector.nameLookupPossibleInAdvance()).andReturn(false).anyTimes();
+    EasyMock.expect(selector.idLookup()).andReturn(lookup).anyTimes();
+    EasyMock.expect(lookup.lookupId("any value")).andReturn(1).anyTimes();
+    EasyMock.expect(lookup.lookupId("another value")).andReturn(-1).anyTimes();
+    EasyMock.replay(selector, lookup);
+
+    VectorValueMatcherFactory matcherFactory =
+        VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor(selector);
+
+    Assert.assertTrue(matcherFactory instanceof SingleValueStringVectorValueMatcher);
+
+    // value exists in column nonboolean matcher
+    VectorValueMatcher matcher = matcherFactory.makeMatcher("any value");
+    Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher);
+    Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize());
+    Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize());
+
+    // value not exist in dictionary uses boolean matcher
+    VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher("another value");
+    Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher);
+    Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize());
+    Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize());
+    EasyMock.verify(selector, lookup);
+  }
+
+  @Test
+  public void testSingleValueStringZeroCardinalityAlwaysBooleanMatcher()
+  {
+    // cardinality 0 has special path to always use boolean matcher
+    SingleValueDimensionVectorSelector selector =
+        EasyMock.createMock(SingleValueDimensionVectorSelector.class);
+    EasyMock.expect(selector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes();
+    EasyMock.expect(selector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes();
+    EasyMock.expect(selector.getValueCardinality()).andReturn(0).anyTimes();
+    EasyMock.replay(selector);
+
+    VectorValueMatcherFactory matcherFactory =
+        VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor(selector);
+
+    Assert.assertTrue(matcherFactory instanceof SingleValueStringVectorValueMatcher);
+
+    VectorValueMatcher matcher = matcherFactory.makeMatcher("any value");
+    Assert.assertTrue(matcher instanceof BooleanVectorValueMatcher);

Review comment:
       If value cardinality == 0, shouldn't it return this thing...
        ```
    // Evaluate "lookupName" and "predicate" on every row.
         return new BaseVectorValueMatcher(selector)
         {
           final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
   
           @Override
           public ReadableVectorMatch match(final ReadableVectorMatch mask)
           {
             final int[] vector = selector.getRowVector();
             final int[] selection = match.getSelection();
   
             int numRows = 0;
   
             for (int i = 0; i < mask.getSelectionSize(); i++) {
               final int rowNum = mask.getSelection()[i];
               if (predicate.apply(selector.lookupName(vector[rowNum]))) {
                 selection[numRows++] = rowNum;
               }
             }
   
             match.setSelectionSize(numRows);
             assert match.isValid(mask);
             return match;
           }
         };
   ```




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