You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/07/27 01:24:32 UTC

svn commit: r1507513 - in /hive/branches/vectorization/ql/src: java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/ java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/ test/org/apache/hadoop/hive/ql/exec/vector/expressions/

Author: hashutosh
Date: Fri Jul 26 23:24:31 2013
New Revision: 1507513

URL: http://svn.apache.org/r1507513
Log:
HIVE-4922 : create template for string scalar compared with string column (Eric Hanson via Ashutosh Chauhan)

Added:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarEqualStringColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterEqualStringColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterStringColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessEqualStringColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessStringColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarNotEqualStringColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/FilterStringScalarCompareColumn.txt
Modified:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarEqualStringColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarEqualStringColumn.java?rev=1507513&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarEqualStringColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarEqualStringColumn.java Fri Jul 26 23:24:31 2013
@@ -0,0 +1,140 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a == comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class FilterStringScalarEqualStringColumn extends VectorExpression {
+  private int colNum;
+  private byte[] value;
+
+  public FilterStringScalarEqualStringColumn(int colNum, byte[] value) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) == 0)) {
+
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) == 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) == 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) == 0)) {
+
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) == 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterEqualStringColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterEqualStringColumn.java?rev=1507513&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterEqualStringColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterEqualStringColumn.java Fri Jul 26 23:24:31 2013
@@ -0,0 +1,140 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a >= comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class FilterStringScalarGreaterEqualStringColumn extends VectorExpression {
+  private int colNum;
+  private byte[] value;
+
+  public FilterStringScalarGreaterEqualStringColumn(int colNum, byte[] value) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) >= 0)) {
+
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) >= 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) >= 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) >= 0)) {
+
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) >= 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterStringColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterStringColumn.java?rev=1507513&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterStringColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarGreaterStringColumn.java Fri Jul 26 23:24:31 2013
@@ -0,0 +1,140 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a > comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class FilterStringScalarGreaterStringColumn extends VectorExpression {
+  private int colNum;
+  private byte[] value;
+
+  public FilterStringScalarGreaterStringColumn(int colNum, byte[] value) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) > 0)) {
+
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) > 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) > 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) > 0)) {
+
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) > 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessEqualStringColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessEqualStringColumn.java?rev=1507513&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessEqualStringColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessEqualStringColumn.java Fri Jul 26 23:24:31 2013
@@ -0,0 +1,140 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a <= comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class FilterStringScalarLessEqualStringColumn extends VectorExpression {
+  private int colNum;
+  private byte[] value;
+
+  public FilterStringScalarLessEqualStringColumn(int colNum, byte[] value) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) <= 0)) {
+
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <= 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <= 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) <= 0)) {
+
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <= 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessStringColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessStringColumn.java?rev=1507513&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessStringColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarLessStringColumn.java Fri Jul 26 23:24:31 2013
@@ -0,0 +1,140 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a < comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class FilterStringScalarLessStringColumn extends VectorExpression {
+  private int colNum;
+  private byte[] value;
+
+  public FilterStringScalarLessStringColumn(int colNum, byte[] value) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) < 0)) {
+
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) < 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) < 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) < 0)) {
+
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) < 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarNotEqualStringColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarNotEqualStringColumn.java?rev=1507513&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarNotEqualStringColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringScalarNotEqualStringColumn.java Fri Jul 26 23:24:31 2013
@@ -0,0 +1,140 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a != comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class FilterStringScalarNotEqualStringColumn extends VectorExpression {
+  private int colNum;
+  private byte[] value;
+
+  public FilterStringScalarNotEqualStringColumn(int colNum, byte[] value) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) != 0)) {
+
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) != 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) != 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) != 0)) {
+
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) != 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) != 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java?rev=1507513&r1=1507512&r2=1507513&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java Fri Jul 26 23:24:31 2013
@@ -151,6 +151,13 @@ public class CodeGen {
       {"FilterStringColumnCompareScalar", "LessEqual", "<="},
       {"FilterStringColumnCompareScalar", "Greater", ">"},
       {"FilterStringColumnCompareScalar", "GreaterEqual", ">="},
+      
+      {"FilterStringScalarCompareColumn", "Equal", "=="},
+      {"FilterStringScalarCompareColumn", "NotEqual", "!="},
+      {"FilterStringScalarCompareColumn", "Less", "<"},
+      {"FilterStringScalarCompareColumn", "LessEqual", "<="},
+      {"FilterStringScalarCompareColumn", "Greater", ">"},
+      {"FilterStringScalarCompareColumn", "GreaterEqual", ">="},
 
       {"FilterStringColumnCompareColumn", "Equal", "=="},
       {"FilterStringColumnCompareColumn", "NotEqual", "!="},
@@ -289,6 +296,8 @@ public class CodeGen {
         generateVectorUDAFVar(tdesc);
       } else if (tdesc[0].equals("FilterStringColumnCompareScalar")) {
         generateFilterStringColumnCompareScalar(tdesc);
+      } else if (tdesc[0].equals("FilterStringScalarCompareColumn")) {
+        generateFilterStringScalarCompareColumn(tdesc);
       } else if (tdesc[0].equals("FilterStringColumnCompareColumn")) {
         generateFilterStringColumnCompareColumn(tdesc);
       } else {
@@ -398,6 +407,13 @@ public class CodeGen {
     writeFile(outputFile, templateString);
   }
 
+  private void generateFilterStringScalarCompareColumn(String[] tdesc) throws IOException {
+    String operatorName = tdesc[1];
+    String className = "FilterStringScalar" + operatorName + "StringColumn";
+    
+    // Template expansion logic is the same for both column-scalar and scalar-column cases.
+    generateFilterStringColumnCompareScalar(tdesc, className);
+  } 
 
   private void generateFilterStringColumnCompareScalar(String[] tdesc) throws IOException {
     String operatorName = tdesc[1];

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/FilterStringScalarCompareColumn.txt
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/FilterStringScalarCompareColumn.txt?rev=1507513&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/FilterStringScalarCompareColumn.txt (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/FilterStringScalarCompareColumn.txt Fri Jul 26 23:24:31 2013
@@ -0,0 +1,140 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class <ClassName> extends VectorExpression {
+  private int colNum;
+  private byte[] value;
+
+  public <ClassName>(int colNum, byte[] value) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) <OperatorSymbol> 0)) {
+
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+      
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) <OperatorSymbol> 0)) {
+
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Modified: hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java?rev=1507513&r1=1507512&r2=1507513&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java (original)
+++ hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java Fri Jul 26 23:24:31 2013
@@ -30,6 +30,9 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringColGreaterEqualStringScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringColLessStringColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringColLessStringScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringScalarEqualStringColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringScalarGreaterStringColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringScalarLessEqualStringColumn;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
@@ -168,6 +171,36 @@ public class TestVectorStringExpressions
   }
 
   @Test
+  // Test string literal to string column comparison
+  public void testStringScalarCompareStringCol() {
+    VectorizedRowBatch batch = makeStringBatch();
+    VectorExpression expr;
+    expr = new FilterStringScalarEqualStringColumn(0, red2);
+    expr.evaluate(batch);
+
+    // only red qualifies, and it's in entry 0
+    Assert.assertTrue(batch.size == 1);
+    Assert.assertTrue(batch.selected[0] == 0);
+
+    batch = makeStringBatch();
+    expr = new FilterStringScalarGreaterStringColumn(0, red2);
+    expr.evaluate(batch);
+
+    // only green qualifies, and it's in entry 1
+    Assert.assertTrue(batch.size == 1);
+    Assert.assertTrue(batch.selected[0] == 1);
+
+    batch = makeStringBatch();
+    expr = new FilterStringScalarLessEqualStringColumn(0, green);
+    expr.evaluate(batch);
+
+    // green and red qualify
+    Assert.assertTrue(batch.size == 2);
+    Assert.assertTrue(batch.selected[0] == 0);
+    Assert.assertTrue(batch.selected[1] == 1);
+  }
+  
+  @Test
   public void testStringColCompareStringColFilter() {
     VectorizedRowBatch batch;
     VectorExpression expr;