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/05/09 06:44:18 UTC

svn commit: r1480527 [1/2] - 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: Thu May  9 04:44:17 2013
New Revision: 1480527

URL: http://svn.apache.org/r1480527
Log:
HIVE-4493 : Implement vectorized filter for string column compared to string column (Eric Hanson via Ashutosh Chauhan)

Added:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColEqualStringCol.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterEqualStringCol.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterStringCol.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessEqualStringCol.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessStringCol.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColNotEqualStringCol.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/FilterStringColumnCompareColumn.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/FilterStringColEqualStringCol.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColEqualStringCol.java?rev=1480527&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColEqualStringCol.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColEqualStringCol.java Thu May  9 04:44:17 2013
@@ -0,0 +1,455 @@
+/**
+ * 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;
+
+/**
+ * Filter the rows in a batch by comparing one string column to another. 
+ * This code is generated from a template.
+ */
+public class FilterStringColEqualStringCol extends VectorExpression {
+  private int colNum1;
+  private int colNum2;
+
+  public FilterStringColEqualStringCol(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    BytesColumnVector inputColVector1 = (BytesColumnVector) batch.cols[colNum1];
+    BytesColumnVector inputColVector2 = (BytesColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    byte[][] vector1 = inputColVector1.vector;
+    byte[][] vector2 = inputColVector2.vector;
+    int[] start1 = inputColVector1.start;
+    int[] start2 = inputColVector2.start;
+    int[] length1 = inputColVector1.length;
+    int[] length2 = inputColVector2.length;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    // handle case where neither input has nulls
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      
+        /* Either all must remain selected or all will be eliminated.
+         * Repeating property will not change.
+         */
+        if (!(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                 vector2[0], start2[0], length2[0]) == 0)) {
+          batch.size = 0;
+        }      
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) == 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) == 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    
+    // handle case where only input 2 has nulls
+    } else if (inputColVector1.noNulls) { 
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) == 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         
+         // no need to check for nulls in input 1
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+        
+          // no values will qualify because every comparison will be with NULL
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+      
+    // handle case where only input 1 has nulls
+    } else if (inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) == 0)) {
+          batch.size = 0; 
+          return;
+        } 
+      } else if (inputColVector1.isRepeating) {
+        if (nullPos1[0]) {
+        
+          // if repeating value is null then every comparison will fail so nothing qualifies
+          batch.size = 0;
+          return; 
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) == 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+            
+    // handle case where both inputs have nulls
+    } else {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] || nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) == 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         if (nullPos1[0]) {
+           batch.size = 0;
+           return;
+         }
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) == 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      } 
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterEqualStringCol.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterEqualStringCol.java?rev=1480527&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterEqualStringCol.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterEqualStringCol.java Thu May  9 04:44:17 2013
@@ -0,0 +1,455 @@
+/**
+ * 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;
+
+/**
+ * Filter the rows in a batch by comparing one string column to another. 
+ * This code is generated from a template.
+ */
+public class FilterStringColGreaterEqualStringCol extends VectorExpression {
+  private int colNum1;
+  private int colNum2;
+
+  public FilterStringColGreaterEqualStringCol(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    BytesColumnVector inputColVector1 = (BytesColumnVector) batch.cols[colNum1];
+    BytesColumnVector inputColVector2 = (BytesColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    byte[][] vector1 = inputColVector1.vector;
+    byte[][] vector2 = inputColVector2.vector;
+    int[] start1 = inputColVector1.start;
+    int[] start2 = inputColVector2.start;
+    int[] length1 = inputColVector1.length;
+    int[] length2 = inputColVector2.length;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    // handle case where neither input has nulls
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      
+        /* Either all must remain selected or all will be eliminated.
+         * Repeating property will not change.
+         */
+        if (!(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                 vector2[0], start2[0], length2[0]) >= 0)) {
+          batch.size = 0;
+        }      
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) >= 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) >= 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    
+    // handle case where only input 2 has nulls
+    } else if (inputColVector1.noNulls) { 
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) >= 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         
+         // no need to check for nulls in input 1
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+        
+          // no values will qualify because every comparison will be with NULL
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+      
+    // handle case where only input 1 has nulls
+    } else if (inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) >= 0)) {
+          batch.size = 0; 
+          return;
+        } 
+      } else if (inputColVector1.isRepeating) {
+        if (nullPos1[0]) {
+        
+          // if repeating value is null then every comparison will fail so nothing qualifies
+          batch.size = 0;
+          return; 
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) >= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+            
+    // handle case where both inputs have nulls
+    } else {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] || nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) >= 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         if (nullPos1[0]) {
+           batch.size = 0;
+           return;
+         }
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) >= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      } 
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterStringCol.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterStringCol.java?rev=1480527&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterStringCol.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColGreaterStringCol.java Thu May  9 04:44:17 2013
@@ -0,0 +1,455 @@
+/**
+ * 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;
+
+/**
+ * Filter the rows in a batch by comparing one string column to another. 
+ * This code is generated from a template.
+ */
+public class FilterStringColGreaterStringCol extends VectorExpression {
+  private int colNum1;
+  private int colNum2;
+
+  public FilterStringColGreaterStringCol(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    BytesColumnVector inputColVector1 = (BytesColumnVector) batch.cols[colNum1];
+    BytesColumnVector inputColVector2 = (BytesColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    byte[][] vector1 = inputColVector1.vector;
+    byte[][] vector2 = inputColVector2.vector;
+    int[] start1 = inputColVector1.start;
+    int[] start2 = inputColVector2.start;
+    int[] length1 = inputColVector1.length;
+    int[] length2 = inputColVector2.length;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    // handle case where neither input has nulls
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      
+        /* Either all must remain selected or all will be eliminated.
+         * Repeating property will not change.
+         */
+        if (!(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                 vector2[0], start2[0], length2[0]) > 0)) {
+          batch.size = 0;
+        }      
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) > 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) > 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    
+    // handle case where only input 2 has nulls
+    } else if (inputColVector1.noNulls) { 
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) > 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         
+         // no need to check for nulls in input 1
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+        
+          // no values will qualify because every comparison will be with NULL
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+      
+    // handle case where only input 1 has nulls
+    } else if (inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) > 0)) {
+          batch.size = 0; 
+          return;
+        } 
+      } else if (inputColVector1.isRepeating) {
+        if (nullPos1[0]) {
+        
+          // if repeating value is null then every comparison will fail so nothing qualifies
+          batch.size = 0;
+          return; 
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) > 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+            
+    // handle case where both inputs have nulls
+    } else {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] || nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) > 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         if (nullPos1[0]) {
+           batch.size = 0;
+           return;
+         }
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) > 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      } 
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessEqualStringCol.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessEqualStringCol.java?rev=1480527&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessEqualStringCol.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessEqualStringCol.java Thu May  9 04:44:17 2013
@@ -0,0 +1,455 @@
+/**
+ * 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;
+
+/**
+ * Filter the rows in a batch by comparing one string column to another. 
+ * This code is generated from a template.
+ */
+public class FilterStringColLessEqualStringCol extends VectorExpression {
+  private int colNum1;
+  private int colNum2;
+
+  public FilterStringColLessEqualStringCol(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    BytesColumnVector inputColVector1 = (BytesColumnVector) batch.cols[colNum1];
+    BytesColumnVector inputColVector2 = (BytesColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    byte[][] vector1 = inputColVector1.vector;
+    byte[][] vector2 = inputColVector2.vector;
+    int[] start1 = inputColVector1.start;
+    int[] start2 = inputColVector2.start;
+    int[] length1 = inputColVector1.length;
+    int[] length2 = inputColVector2.length;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    // handle case where neither input has nulls
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      
+        /* Either all must remain selected or all will be eliminated.
+         * Repeating property will not change.
+         */
+        if (!(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                 vector2[0], start2[0], length2[0]) <= 0)) {
+          batch.size = 0;
+        }      
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) <= 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) <= 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    
+    // handle case where only input 2 has nulls
+    } else if (inputColVector1.noNulls) { 
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) <= 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         
+         // no need to check for nulls in input 1
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+        
+          // no values will qualify because every comparison will be with NULL
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+      
+    // handle case where only input 1 has nulls
+    } else if (inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) <= 0)) {
+          batch.size = 0; 
+          return;
+        } 
+      } else if (inputColVector1.isRepeating) {
+        if (nullPos1[0]) {
+        
+          // if repeating value is null then every comparison will fail so nothing qualifies
+          batch.size = 0;
+          return; 
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) <= 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+            
+    // handle case where both inputs have nulls
+    } else {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] || nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) <= 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         if (nullPos1[0]) {
+           batch.size = 0;
+           return;
+         }
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <= 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      } 
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessStringCol.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessStringCol.java?rev=1480527&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessStringCol.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterStringColLessStringCol.java Thu May  9 04:44:17 2013
@@ -0,0 +1,455 @@
+/**
+ * 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;
+
+/**
+ * Filter the rows in a batch by comparing one string column to another. 
+ * This code is generated from a template.
+ */
+public class FilterStringColLessStringCol extends VectorExpression {
+  private int colNum1;
+  private int colNum2;
+
+  public FilterStringColLessStringCol(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    BytesColumnVector inputColVector1 = (BytesColumnVector) batch.cols[colNum1];
+    BytesColumnVector inputColVector2 = (BytesColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    byte[][] vector1 = inputColVector1.vector;
+    byte[][] vector2 = inputColVector2.vector;
+    int[] start1 = inputColVector1.start;
+    int[] start2 = inputColVector2.start;
+    int[] length1 = inputColVector1.length;
+    int[] length2 = inputColVector2.length;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    // handle case where neither input has nulls
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      
+        /* Either all must remain selected or all will be eliminated.
+         * Repeating property will not change.
+         */
+        if (!(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                 vector2[0], start2[0], length2[0]) < 0)) {
+          batch.size = 0;
+        }      
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) < 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) < 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    
+    // handle case where only input 2 has nulls
+    } else if (inputColVector1.noNulls) { 
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) < 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         
+         // no need to check for nulls in input 1
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+        
+          // no values will qualify because every comparison will be with NULL
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+      
+    // handle case where only input 1 has nulls
+    } else if (inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) < 0)) {
+          batch.size = 0; 
+          return;
+        } 
+      } else if (inputColVector1.isRepeating) {
+        if (nullPos1[0]) {
+        
+          // if repeating value is null then every comparison will fail so nothing qualifies
+          batch.size = 0;
+          return; 
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) < 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      }
+            
+    // handle case where both inputs have nulls
+    } else {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] || nullPos2[0] ||
+            !(StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) < 0)) {
+          batch.size = 0; 
+        } 
+      } else if (inputColVector1.isRepeating) {
+         if (nullPos1[0]) {
+           batch.size = 0;
+           return;
+         }
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) < 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }      
+      } 
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}