You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by zu...@apache.org on 2016/05/30 23:18:54 UTC

[19/50] [abbrv] incubator-quickstep git commit: Added support for the substring function. (#211)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index 72fa9ef..71e4332 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -104,67 +104,69 @@ extern int quickstep_yydebug;
     TOKEN_FALSE = 314,
     TOKEN_FIRST = 315,
     TOKEN_FLOAT = 316,
-    TOKEN_FOREIGN = 317,
-    TOKEN_FROM = 318,
-    TOKEN_FULL = 319,
-    TOKEN_GROUP = 320,
-    TOKEN_HASH = 321,
-    TOKEN_HAVING = 322,
-    TOKEN_HOUR = 323,
-    TOKEN_IN = 324,
-    TOKEN_INDEX = 325,
-    TOKEN_INNER = 326,
-    TOKEN_INSERT = 327,
-    TOKEN_INTEGER = 328,
-    TOKEN_INTERVAL = 329,
-    TOKEN_INTO = 330,
-    TOKEN_JOIN = 331,
-    TOKEN_KEY = 332,
-    TOKEN_LAST = 333,
-    TOKEN_LEFT = 334,
-    TOKEN_LIMIT = 335,
-    TOKEN_LONG = 336,
-    TOKEN_MINUTE = 337,
-    TOKEN_MONTH = 338,
-    TOKEN_NULL = 339,
-    TOKEN_NULLS = 340,
-    TOKEN_OFF = 341,
-    TOKEN_ON = 342,
-    TOKEN_ORDER = 343,
-    TOKEN_OUTER = 344,
-    TOKEN_PARTITION = 345,
-    TOKEN_PARTITIONS = 346,
-    TOKEN_PERCENT = 347,
-    TOKEN_PRIMARY = 348,
-    TOKEN_QUIT = 349,
-    TOKEN_RANGE = 350,
-    TOKEN_REAL = 351,
-    TOKEN_REFERENCES = 352,
-    TOKEN_RIGHT = 353,
-    TOKEN_ROW_DELIMITER = 354,
-    TOKEN_SECOND = 355,
-    TOKEN_SELECT = 356,
-    TOKEN_SET = 357,
-    TOKEN_SMA = 358,
-    TOKEN_SMALLINT = 359,
-    TOKEN_TABLE = 360,
-    TOKEN_THEN = 361,
-    TOKEN_TIME = 362,
-    TOKEN_TIMESTAMP = 363,
-    TOKEN_TRUE = 364,
-    TOKEN_TUPLESAMPLE = 365,
-    TOKEN_UNIQUE = 366,
-    TOKEN_UPDATE = 367,
-    TOKEN_USING = 368,
-    TOKEN_VALUES = 369,
-    TOKEN_VARCHAR = 370,
-    TOKEN_WHEN = 371,
-    TOKEN_WHERE = 372,
-    TOKEN_WITH = 373,
-    TOKEN_YEAR = 374,
-    TOKEN_YEARMONTH = 375,
-    TOKEN_EOF = 376,
-    TOKEN_LEX_ERROR = 377
+    TOKEN_FOR = 317,
+    TOKEN_FOREIGN = 318,
+    TOKEN_FROM = 319,
+    TOKEN_FULL = 320,
+    TOKEN_GROUP = 321,
+    TOKEN_HASH = 322,
+    TOKEN_HAVING = 323,
+    TOKEN_HOUR = 324,
+    TOKEN_IN = 325,
+    TOKEN_INDEX = 326,
+    TOKEN_INNER = 327,
+    TOKEN_INSERT = 328,
+    TOKEN_INTEGER = 329,
+    TOKEN_INTERVAL = 330,
+    TOKEN_INTO = 331,
+    TOKEN_JOIN = 332,
+    TOKEN_KEY = 333,
+    TOKEN_LAST = 334,
+    TOKEN_LEFT = 335,
+    TOKEN_LIMIT = 336,
+    TOKEN_LONG = 337,
+    TOKEN_MINUTE = 338,
+    TOKEN_MONTH = 339,
+    TOKEN_NULL = 340,
+    TOKEN_NULLS = 341,
+    TOKEN_OFF = 342,
+    TOKEN_ON = 343,
+    TOKEN_ORDER = 344,
+    TOKEN_OUTER = 345,
+    TOKEN_PARTITION = 346,
+    TOKEN_PARTITIONS = 347,
+    TOKEN_PERCENT = 348,
+    TOKEN_PRIMARY = 349,
+    TOKEN_QUIT = 350,
+    TOKEN_RANGE = 351,
+    TOKEN_REAL = 352,
+    TOKEN_REFERENCES = 353,
+    TOKEN_RIGHT = 354,
+    TOKEN_ROW_DELIMITER = 355,
+    TOKEN_SECOND = 356,
+    TOKEN_SELECT = 357,
+    TOKEN_SET = 358,
+    TOKEN_SMA = 359,
+    TOKEN_SMALLINT = 360,
+    TOKEN_SUBSTRING = 361,
+    TOKEN_TABLE = 362,
+    TOKEN_THEN = 363,
+    TOKEN_TIME = 364,
+    TOKEN_TIMESTAMP = 365,
+    TOKEN_TRUE = 366,
+    TOKEN_TUPLESAMPLE = 367,
+    TOKEN_UNIQUE = 368,
+    TOKEN_UPDATE = 369,
+    TOKEN_USING = 370,
+    TOKEN_VALUES = 371,
+    TOKEN_VARCHAR = 372,
+    TOKEN_WHEN = 373,
+    TOKEN_WHERE = 374,
+    TOKEN_WITH = 375,
+    TOKEN_YEAR = 376,
+    TOKEN_YEARMONTH = 377,
+    TOKEN_EOF = 378,
+    TOKEN_LEX_ERROR = 379
   };
 #endif
 
@@ -265,7 +267,7 @@ union YYSTYPE
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 269 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 271 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index db2a8af..5959879 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -120,6 +120,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
                       quickstep_types_operations_unaryoperations_DateExtractOperation
+                      quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_utility_Macros
                       quickstep_utility_PtrList

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 2667ee9..78985a0 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -115,6 +115,7 @@
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
 #include "types/operations/unary_operations/DateExtractOperation.hpp"
+#include "types/operations/unary_operations/SubstringOperation.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 #include "utility/PtrList.hpp"
 #include "utility/PtrVector.hpp"
@@ -2068,6 +2069,37 @@ E::ScalarPtr Resolver::resolveExpression(
 
       return E::UnaryExpression::Create(op, argument);
     }
+    case ParseExpression::kSubstring: {
+      const ParseSubstringFunction &parse_substring =
+          static_cast<const ParseSubstringFunction&>(parse_expression);
+
+      // Validate start position and substring length.
+      if (parse_substring.start_position() <= 0) {
+        THROW_SQL_ERROR_AT(&parse_expression)
+            << "The start position must be greater than 0";
+      }
+      if (parse_substring.length() <= 0) {
+        THROW_SQL_ERROR_AT(&parse_expression)
+            << "The substring length must be greater than 0";
+      }
+
+      // Convert 1-base position to 0-base position
+      const std::size_t zero_base_start_position = parse_substring.start_position() - 1;
+      const SubstringOperation &op =
+          SubstringOperation::Instance(zero_base_start_position,
+                                       parse_substring.length());
+
+      const E::ScalarPtr argument =
+          resolveExpression(*parse_substring.operand(),
+                            op.pushDownTypeHint(type_hint),
+                            expression_resolution_info);
+      if (!op.canApplyToType(argument->getValueType())) {
+        THROW_SQL_ERROR_AT(&parse_substring)
+            << "Can not apply substring function to argument of type "
+            << argument->getValueType().getName();
+      }
+      return E::UnaryExpression::Create(op, argument);
+    }
     default:
       LOG(FATAL) << "Unknown scalar type: "
                  << parse_expression.getExpressionType();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/query_optimizer/tests/execution_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Select.test b/query_optimizer/tests/execution_generator/Select.test
index 390b7b6..023ad2e 100644
--- a/query_optimizer/tests/execution_generator/Select.test
+++ b/query_optimizer/tests/execution_generator/Select.test
@@ -765,6 +765,30 @@ WHERE
 +-----------+
 ==
 
+# SUBSTRING function
+SELECT char_col,
+       SUBSTRING(char_col FROM 1 FOR 2) AS negative_value
+FROM test
+WHERE SUBSTRING(char_col FROM 1 FOR 1) = '-'
+--
++--------------------+--------------+
+|char_col            |negative_value|
++--------------------+--------------+
+|         -1 1.000000|            -1|
+|         -3 1.732051|            -3|
+|         -5 2.236068|            -5|
+|         -7 2.645751|            -7|
+|         -9 3.000000|            -9|
+|        -11 3.316625|            -1|
+|        -13 3.605551|            -1|
+|        -15 3.872983|            -1|
+|        -17 4.123106|            -1|
+|        -19 4.358899|            -1|
+|        -21 4.582576|            -2|
+|        -23 4.795832|            -2|
++--------------------+--------------+
+==
+
 # IN predicate
 SELECT *
 FROM generate_series(1, 5) AS gs(i)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/query_optimizer/tests/resolver/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Select.test b/query_optimizer/tests/resolver/Select.test
index 9897934..141bfa0 100644
--- a/query_optimizer/tests/resolver/Select.test
+++ b/query_optimizer/tests/resolver/Select.test
@@ -3048,3 +3048,81 @@ TopLevelPlan
 +-output_attributes=
   +-AttributeReference[id=5,name=x,relation=,type=Int]
   +-AttributeReference[id=6,name=y,relation=,type=Int]
+==
+
+SELECT SUBSTRING(char_col FROM 1 FOR 2)
+FROM test;
+--
+TopLevelPlan
++-plan=Project
+| +-input=TableReference[relation_name=Test,relation_alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=SUBSTRING(char_col FROM 1 FOR 2),relation=,
+|     type=Char(2)]
+|     +-Substring
+|       +-Operand=AttributeReference[id=4,name=char_col,relation=test,
+|         type=Char(20)]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=SUBSTRING(char_col FROM 1 FOR 2),
+    relation=,type=Char(2)]
+==
+
+SELECT *
+FROM test
+WHERE SUBSTRING(vchar_col FROM 1 FOR 2) IN ('12', '34', '56');
+--
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-filter_predicate=InValueList
+| |   +-test_expression=Substring
+| |   | +-Operand=AttributeReference[id=5,name=vchar_col,relation=test,
+| |   |   type=VarChar(20) NULL]
+| |   +-match_expressions=
+| |     +-Literal[value=12,type=Char(2)]
+| |     +-Literal[value=34,type=Char(2)]
+| |     +-Literal[value=56,type=Char(2)]
+| +-project_list=
+|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+|   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+|   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+|   +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
++-output_attributes=
+  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+  +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+  +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+  +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+  +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+==
+
+SELECT SUBSTRING(char_col FROM 0 FOR 2)
+FROM test;
+--
+ERROR: The start position must be greater than 0 (1 : 8)
+SELECT SUBSTRING(char_col FROM 0 FOR ...
+       ^
+==
+
+SELECT SUBSTRING(char_col FROM 1 FOR 0)
+FROM test;
+--
+ERROR: The substring length must be greater than 0 (1 : 8)
+SELECT SUBSTRING(char_col FROM 1 FOR ...
+       ^

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/types/operations/Operation.proto
----------------------------------------------------------------------
diff --git a/types/operations/Operation.proto b/types/operations/Operation.proto
index fb44b1d..33e6b09 100644
--- a/types/operations/Operation.proto
+++ b/types/operations/Operation.proto
@@ -43,6 +43,7 @@ message UnaryOperation {
     NEGATE = 0;
     CAST = 1;
     DATE_EXTRACT = 2;
+    SUBSTRING = 3;
   }
 
   required UnaryOperationID operation_id = 1;
@@ -73,6 +74,14 @@ message DateExtractOperation {
   }
 }
 
+message SubstringOperation {
+  extend UnaryOperation {
+    // Required when operation_id = SUBSTRING.
+    optional int64 start_position = 100;
+    optional int64 substring_length = 101;    
+  }
+}
+
 message BinaryOperation {
   enum BinaryOperationID {
     ADD = 0;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/types/operations/unary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMakeLists.txt b/types/operations/unary_operations/CMakeLists.txt
index 3a50445..5c54d9c 100644
--- a/types/operations/unary_operations/CMakeLists.txt
+++ b/types/operations/unary_operations/CMakeLists.txt
@@ -1,5 +1,7 @@
 #   Copyright 2011-2015 Quickstep Technologies LLC.
 #   Copyright 2015 Pivotal Software, Inc.
+#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+#     University of Wisconsin\u2014Madison.
 #
 #   Licensed under the Apache License, Version 2.0 (the "License");
 #   you may not use this file except in compliance with the License.
@@ -18,6 +20,7 @@ add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
 add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators ../../../empty_src.cpp ArithmeticUnaryOperators.hpp)
 add_library(quickstep_types_operations_unaryoperations_DateExtractOperation DateExtractOperation.cpp DateExtractOperation.hpp)
 add_library(quickstep_types_operations_unaryoperations_NumericCastOperation ../../../empty_src.cpp NumericCastOperation.hpp)
+add_library(quickstep_types_operations_unaryoperations_SubstringOperation SubstringOperation.cpp SubstringOperation.hpp)
 add_library(quickstep_types_operations_unaryoperations_UnaryOperation UnaryOperation.cpp UnaryOperation.hpp)
 add_library(quickstep_types_operations_unaryoperations_UnaryOperationFactory UnaryOperationFactory.cpp UnaryOperationFactory.hpp)
 add_library(quickstep_types_operations_unaryoperations_UnaryOperationID UnaryOperationID.cpp UnaryOperationID.hpp)
@@ -85,6 +88,23 @@ target_link_libraries(quickstep_types_operations_unaryoperations_NumericCastOper
                       quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros
                       quickstep_utility_PtrMap)
+target_link_libraries(quickstep_types_operations_unaryoperations_SubstringOperation
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorUtil
+                      quickstep_types_operations_Operation_proto
+                      quickstep_types_operations_unaryoperations_UnaryOperation
+                      quickstep_types_operations_unaryoperations_UnaryOperationID
+                      quickstep_types_port_strnlen
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros
+                      quickstep_utility_TemplateUtil)
 target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_StorageBlockInfo
@@ -100,6 +120,7 @@ target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperationF
                       quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
                       quickstep_types_operations_unaryoperations_DateExtractOperation
                       quickstep_types_operations_unaryoperations_NumericCastOperation
+                      quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
 
@@ -110,6 +131,7 @@ target_link_libraries(quickstep_types_operations_unaryoperations
                       quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators
                       quickstep_types_operations_unaryoperations_DateExtractOperation
                       quickstep_types_operations_unaryoperations_NumericCastOperation
+                      quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_types_operations_unaryoperations_UnaryOperationFactory
                       quickstep_types_operations_unaryoperations_UnaryOperationID)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/types/operations/unary_operations/SubstringOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.cpp b/types/operations/unary_operations/SubstringOperation.cpp
new file mode 100644
index 0000000..463cd33
--- /dev/null
+++ b/types/operations/unary_operations/SubstringOperation.cpp
@@ -0,0 +1,214 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed 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.
+ **/
+
+#include "types/operations/unary_operations/SubstringOperation.hpp"
+
+#include <algorithm>
+#include <tuple>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorUtil.hpp"
+#include "types/operations/Operation.pb.h"
+#include "types/port/strnlen.hpp"
+#include "utility/TemplateUtil.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+serialization::UnaryOperation SubstringOperation::getProto() const {
+  serialization::UnaryOperation proto;
+  proto.set_operation_id(serialization::UnaryOperation::SUBSTRING);
+  proto.SetExtension(serialization::SubstringOperation::start_position,
+                     start_position_);
+  proto.SetExtension(serialization::SubstringOperation::substring_length,
+                     substring_length_);
+  return proto;
+}
+
+UncheckedUnaryOperator* SubstringOperation::makeUncheckedUnaryOperatorForType(
+    const Type &type) const {
+  DCHECK(type.getSuperTypeID() == Type::kAsciiString);
+
+  const std::size_t input_maximum_length =
+      static_cast<const AsciiStringSuperType&>(type).getStringLength();
+  const bool input_null_terminated = (type.getTypeID() == TypeID::kVarChar);
+
+  const Type *result_type = resultTypeForArgumentType(type);
+  DCHECK(result_type != nullptr);
+
+  return CreateBoolInstantiatedInstance<SubstringUncheckedOperator, UncheckedUnaryOperator>(
+      std::forward_as_tuple(start_position_,
+                            computeMaximumSubstringLength(type),
+                            input_maximum_length,
+                            *result_type),
+      input_null_terminated, type.isNullable());
+}
+
+template <bool null_terminated, bool input_nullable>
+inline void SubstringUncheckedOperator<null_terminated, input_nullable>
+    ::computeSubstring(const char *input,
+                       char *output) const {
+  std::size_t string_length =
+      (null_terminated ? strlen(input) : strnlen(input, maximum_input_length_));
+
+  if (start_position_ >= string_length) {
+    *output = '\0';
+    return;
+  }
+
+  const std::size_t actual_substring_length =
+      std::min(string_length - start_position_, substring_length_);
+  std::memcpy(output, input + start_position_, actual_substring_length);
+
+  if (actual_substring_length < substring_length_) {
+    output[actual_substring_length] = '\0';
+  }
+}
+
+template <bool null_terminated, bool input_nullable>
+TypedValue SubstringUncheckedOperator<null_terminated,
+                                      input_nullable>
+    ::applyToTypedValue(const TypedValue& argument) const {
+  if (input_nullable && argument.isNull()) {
+    return TypedValue(result_type_.getTypeID());
+  }
+
+  char *output_ptr = static_cast<char*>(std::malloc(substring_length_));
+  computeSubstring(static_cast<const char*>(argument.getOutOfLineData()),
+                   output_ptr);
+
+  return TypedValue::CreateWithOwnedData(result_type_.getTypeID(),
+                                         output_ptr,
+                                         substring_length_);
+}
+
+template <bool null_terminated, bool input_nullable>
+TypedValue SubstringUncheckedOperator<null_terminated,
+                                      input_nullable>
+    ::applyToDataPtr(const void *argument) const {
+  if (input_nullable && argument == nullptr) {
+    return TypedValue(result_type_.getTypeID());
+  }
+
+  char *output_ptr = static_cast<char*>(std::malloc(substring_length_));
+  computeSubstring(static_cast<const char*>(argument),
+                   output_ptr);
+
+  return TypedValue::CreateWithOwnedData(result_type_.getTypeID(),
+                                         output_ptr,
+                                         substring_length_);
+}
+
+template <bool null_terminated, bool input_nullable>
+ColumnVector* SubstringUncheckedOperator<null_terminated,
+                                         input_nullable>
+    ::applyToColumnVector(const ColumnVector &argument) const {
+  return InvokeOnColumnVector(
+      argument,
+      [&](const auto &column_vector) -> ColumnVector* {  // NOLINT(build/c++11)
+    NativeColumnVector *result =
+        new NativeColumnVector(result_type_, column_vector.size());
+
+    for (std::size_t cv_pos = 0;
+         cv_pos < column_vector.size();
+         ++cv_pos) {
+      const char *input_ptr = static_cast<const char *>(
+          column_vector.template getUntypedValue<input_nullable>(cv_pos));
+
+      if (input_nullable && input_ptr == nullptr) {
+        result->appendNullValue();
+      } else {
+        this->computeSubstring(input_ptr,
+                               static_cast<char *>(result->getPtrForDirectWrite()));
+      }
+    }
+    return result;
+  });
+}
+
+#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+template <bool null_terminated, bool input_nullable>
+ColumnVector* SubstringUncheckedOperator<null_terminated,
+                                         input_nullable>
+    ::applyToValueAccessor(ValueAccessor *accessor,
+                           const attribute_id argument_attr_id) const {
+  return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+      accessor,
+      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+    NativeColumnVector *result =
+        new NativeColumnVector(result_type_, accessor->getNumTuples());
+
+    accessor->beginIteration();
+    while (accessor->next()) {
+      const char *input_ptr = static_cast<const char *>(
+          accessor->template getUntypedValue<input_nullable>(argument_attr_id));
+
+      if (input_nullable && (input_ptr == nullptr)) {
+        result->appendNullValue();
+      } else {
+        this->computeSubstring(input_ptr,
+                               static_cast<char *>(result->getPtrForDirectWrite()));
+      }
+    }
+    return result;
+  });
+}
+#endif
+
+#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
+template <bool null_terminated, bool input_nullable>
+ColumnVector* SubstringUncheckedOperator<null_terminated,
+                                         input_nullable>
+    ::applyToValueAccessorForJoin(
+        ValueAccessor *accessor,
+        const bool use_left_relation,
+        const attribute_id argument_attr_id,
+        const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
+  return InvokeOnValueAccessorNotAdapter(
+      accessor,
+      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+    NativeColumnVector *result =
+        new NativeColumnVector(result_type_, accessor->getNumTuples());
+
+    for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
+      const char *input_ptr = static_cast<const char *>(
+          accessor->template getUntypedValueAtAbsolutePosition<input_nullable>(
+              argument_attr_id,
+              use_left_relation ? joined_pair.first : joined_pair.second));
+
+      if (input_nullable && input_ptr == nullptr) {
+        result->appendNullValue();
+      } else {
+        this->computeSubstring(input_ptr,
+                               static_cast<char *>(result->getPtrForDirectWrite()));
+      }
+    }
+    return result;
+  });
+}
+#endif
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/types/operations/unary_operations/SubstringOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.hpp b/types/operations/unary_operations/SubstringOperation.hpp
new file mode 100644
index 0000000..d215eae
--- /dev/null
+++ b/types/operations/unary_operations/SubstringOperation.hpp
@@ -0,0 +1,234 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_SUBSTRING_OPERATION_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_SUBSTRING_OPERATION_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdlib>
+#include <cstring>
+#include <memory>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/Operation.pb.h"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "types/operations/unary_operations/UnaryOperationID.hpp"
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class ColumnVector;
+class ValueAccessor;
+
+/**
+ * @brief Operation that extracts a number of characters from a string
+ *        at a given starting position.
+ */
+class SubstringOperation : public UnaryOperation {
+ public:
+  /**
+   * @brief Get a reference to the singleton instance of this Operation for
+   *        the given (start_position, substring_length) pair.
+   **/
+  static const SubstringOperation& Instance(const std::size_t start_position,
+                                            const std::size_t substring_length) {
+    // TODO(jianqiao): This is a temporary solution that creates a new instance
+    // for each distinct pair of start_position and substring_length arguments.
+    // The number of instances may be unbounded if quickstep continuously accepts
+    // queries that call SUBSTRING with different arguments. It still remains to
+    // design a better long-term solution.
+    const auto hash = [](const auto &pair) {
+      return hash_combine_detail::HashCombiner<std::size_t>::CombineHashes(pair.first, pair.second);
+    };
+    static std::unordered_map<std::pair<std::size_t, std::size_t>,
+                              std::unique_ptr<const SubstringOperation>,
+                              decltype(hash)> instance_map(10, hash);
+
+    const std::pair<std::size_t, std::size_t> key_pair =
+        std::make_pair(start_position, substring_length);
+    auto imit = instance_map.find(key_pair);
+    if (imit != instance_map.end()) {
+      return *imit->second;
+    } else {
+      const SubstringOperation *instance =
+          new SubstringOperation(start_position, substring_length);
+      instance_map.emplace(key_pair,
+                           std::unique_ptr<const SubstringOperation>(instance));
+      return *instance;
+    }
+  }
+
+  serialization::UnaryOperation getProto() const override;
+
+  bool canApplyToType(const Type &type) const override {
+    return (type.getSuperTypeID() == Type::kAsciiString);
+  }
+
+  const Type *resultTypeForArgumentType(const Type &type) const override {
+    if (type.getSuperTypeID() == Type::kAsciiString) {
+      // Result is a Char string.
+      return &TypeFactory::GetType(TypeID::kChar,
+                                   computeMaximumSubstringLength(type),
+                                   type.isNullable());
+    }
+    return nullptr;
+  }
+
+  const Type* fixedNullableResultType() const override {
+    // Result type is not fixed (i.e. can have various lengths).
+    return nullptr;
+  }
+
+  bool resultTypeIsPlausible(const Type &result_type) const override {
+    // Result can be coerced to Char or VarChar.
+    return (result_type.getSuperTypeID() == Type::kAsciiString);
+  }
+
+  const Type* pushDownTypeHint(const Type *type_hint) const override {
+    // Input can only be a string, but we don't know the length.
+    return nullptr;
+  }
+
+  TypedValue applyToChecked(const TypedValue &argument,
+                            const Type &argument_type) const override {
+    DCHECK(canApplyToType(argument_type));
+
+    const Type *result_type = resultTypeForArgumentType(argument_type);
+    DCHECK(result_type != nullptr);
+
+    if (argument_type.isNullable() && argument.isNull()) {
+      return result_type->makeNullValue();
+    } else {
+      const std::size_t result_length = computeMaximumSubstringLength(argument_type);
+      char *output_ptr = static_cast<char*>(std::malloc(result_length));
+      const char *input_ptr = static_cast<const char*>(argument.getOutOfLineData());
+
+      const std::size_t string_length = argument.getAsciiStringLength();
+      if (start_position_ >= string_length) {
+        *output_ptr = '\0';
+      } else {
+        const std::size_t actual_substring_length =
+            std::min(string_length - start_position_, substring_length_);
+        std::memcpy(output_ptr, input_ptr + start_position_, actual_substring_length);
+        if (actual_substring_length < result_length) {
+          output_ptr[actual_substring_length] = '\0';
+        }
+      }
+
+      return TypedValue::CreateWithOwnedData(result_type->getTypeID(),
+                                             output_ptr,
+                                             result_length);
+    }
+  }
+
+  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override;
+
+ private:
+  /**
+   * @brief Constructor.
+   *
+   * @param input_type The data type of the input argument for substring.
+   * @param start_position The 0-base starting position of the substring.
+   * @param substring_length The substring length.
+   */
+  SubstringOperation(const std::size_t start_position,
+                     const std::size_t substring_length)
+      : UnaryOperation(UnaryOperationID::kSubstring),
+        start_position_(start_position),
+        substring_length_(substring_length) {
+  }
+
+  /**
+   * @brief Compute an upper bound for the substring length regarding the input
+   *        type and the substring_length_ field.
+   *
+   * @param type The type of the input, must be either CharType or VarCharType.
+   */
+  inline std::size_t computeMaximumSubstringLength(const Type& type) const {
+      DCHECK(type.getSuperTypeID() == Type::kAsciiString);
+
+      // Substring result should have length no greater than the minimum of
+      // (1) the input string length subtract the start position, and
+      // (2) the specified substring length.
+     return std::min(static_cast<const AsciiStringSuperType&>(type).getStringLength() - start_position_,
+                     substring_length_);
+  }
+
+  const std::size_t start_position_;
+  const std::size_t substring_length_;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(SubstringOperation);
+};
+
+template <bool null_terminated, bool input_nullable>
+class SubstringUncheckedOperator : public UncheckedUnaryOperator {
+ public:
+  SubstringUncheckedOperator(const std::size_t start_position,
+                             const std::size_t substring_length,
+                             const std::size_t maximum_input_length,
+                             const Type &result_type)
+      : start_position_(start_position),
+        substring_length_(substring_length),
+        maximum_input_length_(maximum_input_length),
+        result_type_(result_type) {
+  }
+
+  TypedValue applyToTypedValue(const TypedValue& argument) const override;
+
+  TypedValue applyToDataPtr(const void *argument) const override;
+
+  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override;
+
+#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
+                                     const attribute_id argument_attr_id) const override;
+#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+
+#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
+  ColumnVector* applyToValueAccessorForJoin(
+      ValueAccessor *accessor,
+      const bool use_left_relation,
+      const attribute_id argument_attr_id,
+      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
+#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
+
+ private:
+  inline void computeSubstring(const char *input, char *output) const;
+
+  const std::size_t start_position_;
+  const std::size_t substring_length_;
+  const std::size_t maximum_input_length_;
+  const Type &result_type_;
+
+  DISALLOW_COPY_AND_ASSIGN(SubstringUncheckedOperator);
+};
+
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_SUBSTRING_OPERATION_HPP_ */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/types/operations/unary_operations/UnaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperation.cpp b/types/operations/unary_operations/UnaryOperation.cpp
index 0f1f2c7..6721bd6 100644
--- a/types/operations/unary_operations/UnaryOperation.cpp
+++ b/types/operations/unary_operations/UnaryOperation.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -33,6 +35,8 @@ serialization::UnaryOperation UnaryOperation::getProto() const {
       FATAL_ERROR("Must use the overridden NumericCastOperation::getProto");
     case UnaryOperationID::kDateExtract:
       FATAL_ERROR("Must use the overridden DateExtractOperation::getProto");
+    case UnaryOperationID::kSubstring:
+      FATAL_ERROR("Must use the overridden SubstringOperation::getProto");
     default:
       FATAL_ERROR("Unrecognized UnaryOperationID in UnaryOperation::getProto");
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/types/operations/unary_operations/UnaryOperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.cpp b/types/operations/unary_operations/UnaryOperationFactory.cpp
index 9d79cb0..13cb76e 100644
--- a/types/operations/unary_operations/UnaryOperationFactory.cpp
+++ b/types/operations/unary_operations/UnaryOperationFactory.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -24,6 +26,7 @@
 #include "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"
 #include "types/operations/unary_operations/NumericCastOperation.hpp"
 #include "types/operations/unary_operations/DateExtractOperation.hpp"
+#include "types/operations/unary_operations/SubstringOperation.hpp"
 #include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/Macros.hpp"
 
@@ -39,6 +42,8 @@ const UnaryOperation& UnaryOperationFactory::GetUnaryOperation(const UnaryOperat
       FATAL_ERROR("Getting a CastOperation through GetUnaryOperation is not supported");
     case UnaryOperationID::kDateExtract:
       FATAL_ERROR("Getting a DateExtractOperation through GetUnaryOperation is not supported");
+    case UnaryOperationID::kSubstring:
+      FATAL_ERROR("Getting a SubstringOperation through GetUnaryOperation is not supported");
     default:
       FATAL_ERROR("Unknown UnaryOperationID");
   }
@@ -64,6 +69,9 @@ bool UnaryOperationFactory::ProtoIsValid(const serialization::UnaryOperation &pr
     case serialization::UnaryOperation::DATE_EXTRACT:
       return proto.HasExtension(serialization::DateExtractOperation::unit)
           && DateExtractOperation_Unit_IsValid(proto.GetExtension(serialization::DateExtractOperation::unit));
+    case serialization::UnaryOperation::SUBSTRING:
+      return proto.HasExtension(serialization::SubstringOperation::start_position)
+          && proto.HasExtension(serialization::SubstringOperation::substring_length);
     default:
       return false;
   }
@@ -100,6 +108,10 @@ const UnaryOperation& UnaryOperationFactory::ReconstructFromProto(
         default:
           FATAL_ERROR("Unrecognized DateExtractOperation unit in UnaryOperation::ReconstructFromProto");
       }
+    case serialization::UnaryOperation::SUBSTRING:
+      return SubstringOperation::Instance(
+          proto.GetExtension(serialization::SubstringOperation::start_position),
+          proto.GetExtension(serialization::SubstringOperation::substring_length));
     default:
       FATAL_ERROR("Unrecognized UnaryOperationID in UnaryOperation::ReconstructFromProto");
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/types/operations/unary_operations/UnaryOperationID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.cpp b/types/operations/unary_operations/UnaryOperationID.cpp
index c45a0a9..dae64d3 100644
--- a/types/operations/unary_operations/UnaryOperationID.cpp
+++ b/types/operations/unary_operations/UnaryOperationID.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -20,11 +22,11 @@
 namespace quickstep {
 
 const char *kUnaryOperationNames[] = {
-  "Negate", "Cast", "DateExtract"
+  "Negate", "Cast", "DateExtract", "Substring"
 };
 
 const char *kUnaryOperationShortNames[] = {
-  "-", "Cast", "DateExtract"
+  "-", "Cast", "DateExtract", "Substring"
 };
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/767b2ef1/types/operations/unary_operations/UnaryOperationID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.hpp b/types/operations/unary_operations/UnaryOperationID.hpp
index ba3e0ef..9cbb6e4 100644
--- a/types/operations/unary_operations/UnaryOperationID.hpp
+++ b/types/operations/unary_operations/UnaryOperationID.hpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -33,6 +35,7 @@ enum class UnaryOperationID {
   kNegate = 0,
   kCast,
   kDateExtract,
+  kSubstring,
   kNumUnaryOperationIDs  // Not a real UnaryOperationID, exists for counting purposes.
 };