You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2017/10/25 20:02:19 UTC

[1/3] incubator-quickstep git commit: Support Multiple Tuple Inserts

Repository: incubator-quickstep
Updated Branches:
  refs/heads/master 79bfcf9ed -> 0fe838dfe


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index f6b5247..142059d 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -198,6 +198,7 @@ union YYSTYPE
   quickstep::NumericParseLiteralValue *numeric_literal_value_;
   quickstep::ParseLiteralValue *literal_value_;
   quickstep::PtrList<quickstep::ParseScalarLiteral> *literal_value_list_;
+  quickstep::PtrList<quickstep::PtrList<quickstep::ParseScalarLiteral>> *literal_value_list_multiple_;
 
   quickstep::ParseExpression *expression_;
 
@@ -288,7 +289,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 292 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 293 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 372d576..14d8949 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1461,70 +1461,72 @@ void ExecutionGenerator::convertInsertTuple(
       *catalog_database_->getRelationById(
           input_relation_info->relation->getID());
 
-  // Construct the tuple proto to be inserted.
-  const QueryContext::tuple_id tuple_index = query_context_proto_->tuples_size();
+  for (const std::vector<expressions::ScalarLiteralPtr> &tuple : physical_plan->column_values()) {
+    // Construct the tuple proto to be inserted.
+    const QueryContext::tuple_id tuple_index = query_context_proto_->tuples_size();
 
-  S::Tuple *tuple_proto = query_context_proto_->add_tuples();
-  for (const E::ScalarLiteralPtr &literal : physical_plan->column_values()) {
-    tuple_proto->add_attribute_values()->CopyFrom(literal->value().getProto());
-  }
+    S::Tuple *tuple_proto = query_context_proto_->add_tuples();
+    for (const E::ScalarLiteralPtr &literal : tuple) {
+      tuple_proto->add_attribute_values()->CopyFrom(literal->value().getProto());
+    }
 
-  // FIXME(qzeng): A better way is using a traits struct to look up whether a storage
-  //               block supports ad-hoc insertion instead of hard-coding the block types.
-  const StorageBlockLayout &storage_block_layout =
-      input_relation.getDefaultStorageBlockLayout();
-  if (storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
-      TupleStorageSubBlockDescription::COMPRESSED_COLUMN_STORE ||
-      storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
-            TupleStorageSubBlockDescription::COMPRESSED_PACKED_ROW_STORE) {
-    THROW_SQL_ERROR() << "INSERT statement is not supported for the relation "
-                      << input_relation.getName()
-                      << ", because its storage blocks do not support ad-hoc insertion";
-  }
+    // FIXME(qzeng): A better way is using a traits struct to look up whether a storage
+    //               block supports ad-hoc insertion instead of hard-coding the block types.
+    const StorageBlockLayout &storage_block_layout =
+        input_relation.getDefaultStorageBlockLayout();
+    if (storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
+        TupleStorageSubBlockDescription::COMPRESSED_COLUMN_STORE ||
+        storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
+              TupleStorageSubBlockDescription::COMPRESSED_PACKED_ROW_STORE) {
+      THROW_SQL_ERROR() << "INSERT statement is not supported for the relation "
+                        << input_relation.getName()
+                        << ", because its storage blocks do not support ad-hoc insertion";
+    }
 
-  // Create InsertDestination proto.
-  const QueryContext::insert_destination_id insert_destination_index =
-      query_context_proto_->insert_destinations_size();
-  S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
+    // Create InsertDestination proto.
+    const QueryContext::insert_destination_id insert_destination_index =
+        query_context_proto_->insert_destinations_size();
+    S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
 
-  insert_destination_proto->set_relation_id(input_relation.getID());
-  insert_destination_proto->mutable_layout()->MergeFrom(
-      input_relation.getDefaultStorageBlockLayout().getDescription());
+    insert_destination_proto->set_relation_id(input_relation.getID());
+    insert_destination_proto->mutable_layout()->MergeFrom(
+        input_relation.getDefaultStorageBlockLayout().getDescription());
 
-  if (input_relation.hasPartitionScheme()) {
-    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::PARTITION_AWARE);
-    insert_destination_proto->MutableExtension(S::PartitionAwareInsertDestination::partition_scheme)
-        ->MergeFrom(input_relation.getPartitionScheme()->getProto());
-  } else {
-    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
+    if (input_relation.hasPartitionScheme()) {
+      insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::PARTITION_AWARE);
+      insert_destination_proto->MutableExtension(S::PartitionAwareInsertDestination::partition_scheme)
+          ->MergeFrom(input_relation.getPartitionScheme()->getProto());
+    } else {
+      insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
 
-    const vector<block_id> blocks(input_relation.getBlocksSnapshot());
-    for (const block_id block : blocks) {
-      insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+      const vector<block_id> blocks(input_relation.getBlocksSnapshot());
+      for (const block_id block : blocks) {
+        insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+      }
     }
-  }
 
-  const QueryPlan::DAGNodeIndex insert_operator_index =
-      execution_plan_->addRelationalOperator(
-          new InsertOperator(query_handle_->query_id(),
-                             input_relation,
-                             insert_destination_index,
-                             tuple_index));
-  insert_destination_proto->set_relational_op_index(insert_operator_index);
+    const QueryPlan::DAGNodeIndex insert_operator_index =
+        execution_plan_->addRelationalOperator(
+            new InsertOperator(query_handle_->query_id(),
+                               input_relation,
+                               insert_destination_index,
+                               tuple_index));
+    insert_destination_proto->set_relational_op_index(insert_operator_index);
 
-  CatalogRelation *mutable_relation =
-      catalog_database_->getRelationByIdMutable(input_relation.getID());
-  const QueryPlan::DAGNodeIndex save_blocks_index =
-      execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
-  if (!input_relation_info->isStoredRelation()) {
-    execution_plan_->addDirectDependency(insert_operator_index,
-                                         input_relation_info->producer_operator_index,
-                                         true /* is_pipeline_breaker */);
+    CatalogRelation *mutable_relation =
+        catalog_database_->getRelationByIdMutable(input_relation.getID());
+    const QueryPlan::DAGNodeIndex save_blocks_index =
+        execution_plan_->addRelationalOperator(
+            new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
+    if (!input_relation_info->isStoredRelation()) {
+      execution_plan_->addDirectDependency(insert_operator_index,
+                                           input_relation_info->producer_operator_index,
+                                           true /* is_pipeline_breaker */);
+    }
+    execution_plan_->addDirectDependency(save_blocks_index,
+                                         insert_operator_index,
+                                         false /* is_pipeline_breaker */);
   }
-  execution_plan_->addDirectDependency(save_blocks_index,
-                                       insert_operator_index,
-                                       false /* is_pipeline_breaker */);
 }
 
 void ExecutionGenerator::convertInsertSelection(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/logical/InsertTuple.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/InsertTuple.cpp b/query_optimizer/logical/InsertTuple.cpp
index e5ffa35..e2ce196 100644
--- a/query_optimizer/logical/InsertTuple.cpp
+++ b/query_optimizer/logical/InsertTuple.cpp
@@ -41,8 +41,10 @@ void InsertTuple::getFieldStringItems(
   non_container_child_field_names->push_back("input");
   non_container_child_fields->push_back(input_);
 
-  container_child_field_names->push_back("column_values");
-  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(column_values_));
+  for (const auto &column_values : column_values_) {
+    container_child_field_names->push_back("column_values");
+    container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(column_values));
+  }
 }
 
 }  // namespace logical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/logical/InsertTuple.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/InsertTuple.hpp b/query_optimizer/logical/InsertTuple.hpp
index fd0301e..dd35510 100644
--- a/query_optimizer/logical/InsertTuple.hpp
+++ b/query_optimizer/logical/InsertTuple.hpp
@@ -61,7 +61,7 @@ class InsertTuple : public Logical {
   /**
    * @return Column values to be used to compose a new tuple.
    */
-  const std::vector<expressions::ScalarLiteralPtr>& column_values() const {
+  const std::vector<std::vector<expressions::ScalarLiteralPtr>>& column_values() const {
     return column_values_;
   }
 
@@ -83,12 +83,12 @@ class InsertTuple : public Logical {
    * @brief Creates an InsertTuple logical node.
    *
    * @param input The input produces the relation to insert the tuple to.
-   * @param column_values The column values of the tuple to be inserted.
+   * @param column_values The column values of the tuples to be inserted.
    * @return An immutable InsertTuple node.
    */
   static InsertTuplePtr Create(
       const LogicalPtr &input,
-      const std::vector<expressions::ScalarLiteralPtr> &column_values) {
+      const std::vector<std::vector<expressions::ScalarLiteralPtr>> &column_values) {
     return InsertTuplePtr(new InsertTuple(input, column_values));
   }
 
@@ -103,13 +103,13 @@ class InsertTuple : public Logical {
 
  private:
   InsertTuple(const LogicalPtr &input,
-              const std::vector<expressions::ScalarLiteralPtr> &column_values)
+              const std::vector<std::vector<expressions::ScalarLiteralPtr>> &column_values)
       : input_(input), column_values_(column_values) {
     addChild(input_);
   }
 
   LogicalPtr input_;
-  std::vector<expressions::ScalarLiteralPtr> column_values_;
+  std::vector<std::vector<expressions::ScalarLiteralPtr>> column_values_;
 
   DISALLOW_COPY_AND_ASSIGN(InsertTuple);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/physical/InsertTuple.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/InsertTuple.cpp b/query_optimizer/physical/InsertTuple.cpp
index 3085389..b209aa0 100644
--- a/query_optimizer/physical/InsertTuple.cpp
+++ b/query_optimizer/physical/InsertTuple.cpp
@@ -40,8 +40,10 @@ void InsertTuple::getFieldStringItems(
   non_container_child_field_names->push_back("input");
   non_container_child_fields->push_back(input_);
 
-  container_child_field_names->push_back("column_values");
-  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(column_values_));
+  for (const auto &column_values : column_values_) {
+    container_child_field_names->push_back("column_values");
+    container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(column_values));
+  }
 }
 
 }  // namespace physical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/physical/InsertTuple.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/InsertTuple.hpp b/query_optimizer/physical/InsertTuple.hpp
index 40f2582..10c7c5b 100644
--- a/query_optimizer/physical/InsertTuple.hpp
+++ b/query_optimizer/physical/InsertTuple.hpp
@@ -69,7 +69,7 @@ class InsertTuple : public Physical {
   /**
    * @return Column values to be used to compose a new tuple.
    */
-  const std::vector<expressions::ScalarLiteralPtr>& column_values() const {
+  const std::vector<std::vector<expressions::ScalarLiteralPtr>>& column_values() const {
     return column_values_;
   }
 
@@ -103,7 +103,7 @@ class InsertTuple : public Physical {
    */
   static InsertTuplePtr Create(
       const PhysicalPtr &input,
-      const std::vector<expressions::ScalarLiteralPtr> &column_values) {
+      const std::vector<std::vector<expressions::ScalarLiteralPtr>> &column_values) {
     return InsertTuplePtr(new InsertTuple(input, column_values));
   }
 
@@ -118,13 +118,13 @@ class InsertTuple : public Physical {
 
  private:
   InsertTuple(const PhysicalPtr &input,
-              const std::vector<expressions::ScalarLiteralPtr> &column_values)
+              const std::vector<std::vector<expressions::ScalarLiteralPtr>> &column_values)
       : input_(input), column_values_(column_values) {
     addChild(input_);
   }
 
   PhysicalPtr input_;
-  std::vector<expressions::ScalarLiteralPtr> column_values_;
+  std::vector<std::vector<expressions::ScalarLiteralPtr>> column_values_;
 
   DISALLOW_COPY_AND_ASSIGN(InsertTuple);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 2991568..0b6dc22 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -1060,73 +1060,81 @@ L::LogicalPtr Resolver::resolveInsertTuple(
   // Resolve column values.
   const std::vector<E::AttributeReferencePtr> relation_attributes =
       input_logical->getOutputAttributes();
-  const PtrList<ParseScalarLiteral> &parse_column_values =
+  const PtrList<PtrList<ParseScalarLiteral>> &parse_column_values_list =
       insert_statement.getLiteralValues();
-  DCHECK_GT(parse_column_values.size(), 0u);
 
-  if (parse_column_values.size() > relation_attributes.size()) {
-    THROW_SQL_ERROR_AT(insert_statement.relation_name())
-        << "The relation " << insert_statement.relation_name()->value()
-        << " has " << std::to_string(relation_attributes.size())
-        << " columns, but " << std::to_string(parse_column_values.size())
-        << " values are provided";
-  }
+  std::vector<std::vector<E::ScalarLiteralPtr>> resolved_column_values_list;
+  DCHECK_GT(parse_column_values_list.size(), 0u);
 
-  std::vector<E::ScalarLiteralPtr> resolved_column_values;
-  std::vector<E::AttributeReferencePtr>::size_type aid = 0;
-  for (const ParseScalarLiteral &parse_literal_value : parse_column_values) {
-    E::ScalarLiteralPtr resolved_literal_value;
-    ExpressionResolutionInfo expr_resolution_info(
-        name_resolver,
-        "INSERT statement" /* clause_name */,
-        nullptr /* select_list_info */);
-    // When resolving the literal, use the attribute's Type as a hint.
-    CHECK(E::SomeScalarLiteral::MatchesWithConditionalCast(
-        resolveExpression(parse_literal_value,
-                          &(relation_attributes[aid]->getValueType()),
-                          &expr_resolution_info),
-        &resolved_literal_value));
-
-    // Check that the resolved Type is safely coercible to the attribute's
-    // Type.
-    if (!relation_attributes[aid]->getValueType().isSafelyCoercibleFrom(
-            resolved_literal_value->getValueType())) {
-      THROW_SQL_ERROR_AT(&parse_literal_value)
-          << "The assigned value for the column "
-          << relation_attributes[aid]->attribute_name() << " has the type "
-          << resolved_literal_value->getValueType().getName()
-          << ", which cannot be safely coerced to the column's type "
-          << relation_attributes[aid]->getValueType().getName();
-    }
+  for (const PtrList<ParseScalarLiteral> &parse_column_values : parse_column_values_list) {
+    DCHECK_GT(parse_column_values.size(), 0u);
 
-    // If the Type is not exactly right (but is safely coercible), coerce it.
-    if (!resolved_literal_value->getValueType().equals(
-            relation_attributes[aid]->getValueType())) {
-      resolved_literal_value = E::ScalarLiteral::Create(
-          relation_attributes[aid]->getValueType().coerceValue(
-              resolved_literal_value->value(),
-              resolved_literal_value->getValueType()),
-          relation_attributes[aid]->getValueType());
+    if (parse_column_values.size() > relation_attributes.size()) {
+      THROW_SQL_ERROR_AT(insert_statement.relation_name())
+          << "The relation " << insert_statement.relation_name()->value()
+          << " has " << std::to_string(relation_attributes.size())
+          << " columns, but " << std::to_string(parse_column_values.size())
+          << " values are provided";
     }
 
-    resolved_column_values.push_back(resolved_literal_value);
-    ++aid;
-  }
+    std::vector<E::ScalarLiteralPtr> resolved_column_values;
+    std::vector<E::AttributeReferencePtr>::size_type aid = 0;
+    for (const ParseScalarLiteral &parse_literal_value : parse_column_values) {
+      E::ScalarLiteralPtr resolved_literal_value;
+      ExpressionResolutionInfo expr_resolution_info(
+          name_resolver,
+          "INSERT statement" /* clause_name */,
+          nullptr /* select_list_info */);
+      // When resolving the literal, use the attribute's Type as a hint.
+      CHECK(E::SomeScalarLiteral::MatchesWithConditionalCast(
+          resolveExpression(parse_literal_value,
+                            &(relation_attributes[aid]->getValueType()),
+                            &expr_resolution_info),
+          &resolved_literal_value));
+
+      // Check that the resolved Type is safely coercible to the attribute's
+      // Type.
+      if (!relation_attributes[aid]->getValueType().isSafelyCoercibleFrom(
+              resolved_literal_value->getValueType())) {
+        THROW_SQL_ERROR_AT(&parse_literal_value)
+            << "The assigned value for the column "
+            << relation_attributes[aid]->attribute_name() << " has the type "
+            << resolved_literal_value->getValueType().getName()
+            << ", which cannot be safely coerced to the column's type "
+            << relation_attributes[aid]->getValueType().getName();
+      }
 
-  while (aid < relation_attributes.size()) {
-    if (!relation_attributes[aid]->getValueType().isNullable()) {
-      THROW_SQL_ERROR_AT(insert_statement.relation_name())
-          << "Must assign a non-NULL value to column "
-          << relation_attributes[aid]->attribute_name();
+      // If the Type is not exactly right (but is safely coercible), coerce it.
+      if (!resolved_literal_value->getValueType().equals(
+              relation_attributes[aid]->getValueType())) {
+        resolved_literal_value = E::ScalarLiteral::Create(
+            relation_attributes[aid]->getValueType().coerceValue(
+                resolved_literal_value->value(),
+                resolved_literal_value->getValueType()),
+            relation_attributes[aid]->getValueType());
+      }
+
+      resolved_column_values.push_back(resolved_literal_value);
+      ++aid;
+    }
+
+    while (aid < relation_attributes.size()) {
+      if (!relation_attributes[aid]->getValueType().isNullable()) {
+        THROW_SQL_ERROR_AT(insert_statement.relation_name())
+            << "Must assign a non-NULL value to column "
+            << relation_attributes[aid]->attribute_name();
+      }
+      // Create a NULL value.
+      resolved_column_values.push_back(E::ScalarLiteral::Create(
+          relation_attributes[aid]->getValueType().makeNullValue(),
+          relation_attributes[aid]->getValueType()));
+      ++aid;
     }
-    // Create a NULL value.
-    resolved_column_values.push_back(E::ScalarLiteral::Create(
-        relation_attributes[aid]->getValueType().makeNullValue(),
-        relation_attributes[aid]->getValueType()));
-    ++aid;
+
+    resolved_column_values_list.push_back(std::move(resolved_column_values));
   }
 
-  return L::InsertTuple::Create(input_logical, resolved_column_values);
+  return L::InsertTuple::Create(input_logical, resolved_column_values_list);
 }
 
 L::LogicalPtr Resolver::resolveUpdate(


[2/3] incubator-quickstep git commit: Support Multiple Tuple Inserts

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 72c61dd..9b77875 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -341,6 +341,7 @@ union YYSTYPE
   quickstep::NumericParseLiteralValue *numeric_literal_value_;
   quickstep::ParseLiteralValue *literal_value_;
   quickstep::PtrList<quickstep::ParseScalarLiteral> *literal_value_list_;
+  quickstep::PtrList<quickstep::PtrList<quickstep::ParseScalarLiteral>> *literal_value_list_multiple_;
 
   quickstep::ParseExpression *expression_;
 
@@ -431,7 +432,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 435 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 436 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -460,13 +461,13 @@ int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedSta
 #endif /* !YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED  */
 
 /* Copy the second part of user declarations.  */
-#line 222 "../SqlParser.ypp" /* yacc.c:358  */
+#line 223 "../SqlParser.ypp" /* yacc.c:358  */
 
 /* This header needs YYSTYPE, which is defined by the %union directive above */
 #include "SqlLexer_gen.hpp"
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature);
 
-#line 470 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 471 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -710,16 +711,16 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  50
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1391
+#define YYLAST   1327
 
 /* YYNTOKENS -- Number of terminals.  */
 #define YYNTOKENS  148
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  110
+#define YYNNTS  111
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  298
+#define YYNRULES  300
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  550
+#define YYNSTATES  555
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
@@ -779,36 +780,37 @@ static const yytype_uint8 yytranslate[] =
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   640,   640,   644,   648,   652,   656,   659,   666,   669,
-     672,   675,   678,   681,   684,   687,   690,   693,   699,   705,
-     712,   718,   725,   734,   739,   748,   753,   758,   762,   768,
-     773,   776,   779,   784,   787,   790,   793,   796,   799,   802,
-     805,   808,   811,   823,   826,   829,   847,   867,   870,   873,
-     878,   883,   889,   895,   904,   908,   914,   917,   922,   927,
-     932,   939,   946,   950,   956,   959,   964,   967,   972,   975,
-     980,   983,  1002,  1005,  1010,  1014,  1020,  1023,  1026,  1029,
-    1034,  1037,  1040,  1047,  1052,  1063,  1068,  1073,  1077,  1081,
-    1087,  1090,  1096,  1104,  1107,  1110,  1116,  1121,  1126,  1130,
-    1136,  1140,  1143,  1148,  1151,  1156,  1161,  1166,  1170,  1176,
-    1185,  1188,  1193,  1196,  1215,  1220,  1224,  1230,  1236,  1245,
-    1250,  1258,  1264,  1270,  1273,  1276,  1281,  1284,  1289,  1293,
-    1299,  1302,  1305,  1310,  1315,  1320,  1323,  1326,  1331,  1334,
-    1337,  1340,  1343,  1346,  1349,  1352,  1357,  1360,  1365,  1369,
-    1373,  1376,  1380,  1383,  1388,  1391,  1396,  1399,  1404,  1408,
-    1414,  1417,  1422,  1425,  1430,  1433,  1438,  1441,  1460,  1463,
-    1468,  1472,  1478,  1484,  1489,  1492,  1497,  1500,  1505,  1508,
-    1513,  1516,  1521,  1522,  1525,  1530,  1531,  1534,  1539,  1543,
-    1549,  1556,  1559,  1562,  1567,  1570,  1573,  1579,  1582,  1587,
-    1592,  1601,  1606,  1615,  1620,  1623,  1628,  1631,  1636,  1642,
-    1648,  1651,  1654,  1657,  1660,  1663,  1669,  1678,  1681,  1686,
-    1689,  1694,  1697,  1702,  1705,  1708,  1711,  1715,  1719,  1722,
-    1725,  1728,  1731,  1736,  1740,  1744,  1747,  1752,  1757,  1761,
-    1767,  1770,  1775,  1779,  1785,  1790,  1794,  1800,  1805,  1808,
-    1813,  1817,  1823,  1826,  1829,  1832,  1844,  1848,  1867,  1880,
-    1895,  1898,  1901,  1904,  1907,  1910,  1915,  1919,  1925,  1928,
-    1933,  1937,  1944,  1947,  1950,  1953,  1956,  1959,  1962,  1965,
-    1968,  1971,  1976,  1987,  1990,  1995,  1998,  2001,  2007,  2011,
-    2017,  2020,  2028,  2031,  2034,  2037,  2043,  2048,  2053
+       0,   644,   644,   648,   652,   656,   660,   663,   670,   673,
+     676,   679,   682,   685,   688,   691,   694,   697,   703,   709,
+     716,   722,   729,   738,   743,   752,   757,   762,   766,   772,
+     777,   780,   783,   788,   791,   794,   797,   800,   803,   806,
+     809,   812,   815,   827,   830,   833,   851,   871,   874,   877,
+     882,   887,   893,   899,   908,   912,   918,   921,   926,   931,
+     936,   943,   950,   954,   960,   963,   968,   971,   976,   979,
+     984,   987,  1006,  1009,  1014,  1018,  1024,  1027,  1030,  1033,
+    1038,  1041,  1044,  1051,  1056,  1067,  1072,  1077,  1081,  1085,
+    1091,  1094,  1100,  1108,  1111,  1114,  1120,  1125,  1130,  1134,
+    1140,  1144,  1147,  1152,  1155,  1160,  1165,  1170,  1174,  1180,
+    1189,  1192,  1197,  1200,  1219,  1224,  1228,  1234,  1240,  1249,
+    1254,  1262,  1268,  1274,  1277,  1280,  1285,  1288,  1293,  1297,
+    1303,  1306,  1309,  1314,  1319,  1324,  1327,  1330,  1335,  1338,
+    1341,  1344,  1347,  1350,  1353,  1356,  1361,  1364,  1369,  1373,
+    1377,  1380,  1384,  1387,  1392,  1395,  1400,  1403,  1408,  1412,
+    1418,  1421,  1426,  1429,  1434,  1437,  1442,  1445,  1464,  1467,
+    1472,  1476,  1482,  1488,  1493,  1496,  1501,  1504,  1509,  1512,
+    1517,  1520,  1525,  1526,  1529,  1534,  1535,  1538,  1543,  1547,
+    1553,  1560,  1563,  1566,  1571,  1574,  1577,  1583,  1586,  1591,
+    1596,  1605,  1610,  1619,  1624,  1627,  1632,  1635,  1640,  1646,
+    1652,  1655,  1658,  1661,  1664,  1667,  1673,  1682,  1685,  1690,
+    1693,  1698,  1701,  1706,  1709,  1712,  1715,  1719,  1723,  1726,
+    1729,  1732,  1735,  1740,  1744,  1748,  1751,  1756,  1761,  1765,
+    1771,  1774,  1779,  1783,  1789,  1794,  1798,  1804,  1809,  1812,
+    1817,  1821,  1827,  1830,  1833,  1836,  1848,  1852,  1871,  1884,
+    1899,  1902,  1905,  1908,  1911,  1914,  1919,  1923,  1929,  1933,
+    1939,  1942,  1947,  1951,  1958,  1961,  1964,  1967,  1970,  1973,
+    1976,  1979,  1982,  1985,  1990,  2001,  2004,  2009,  2012,  2015,
+    2021,  2025,  2031,  2034,  2042,  2045,  2048,  2051,  2057,  2062,
+    2067
 };
 #endif
 
@@ -883,10 +885,11 @@ static const char *const yytname[] =
   "simple_when_clause_list", "simple_when_clause",
   "searched_when_clause_list", "searched_when_clause", "opt_else_clause",
   "expression_list", "literal_value", "datetime_unit",
-  "literal_value_commalist", "attribute_ref", "attribute_ref_list",
-  "comparison_operation", "unary_operation", "add_operation",
-  "multiply_operation", "name_commalist", "any_name", "boolean_value",
-  "command", "command_argument_list", YY_NULLPTR
+  "literal_value_commalist", "literal_value_commalist_multiple",
+  "attribute_ref", "attribute_ref_list", "comparison_operation",
+  "unary_operation", "add_operation", "multiply_operation",
+  "name_commalist", "any_name", "boolean_value", "command",
+  "command_argument_list", YY_NULLPTR
 };
 #endif
 
@@ -913,10 +916,10 @@ static const yytype_uint16 yytoknum[] =
 };
 # endif
 
-#define YYPACT_NINF -395
+#define YYPACT_NINF -410
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-395)))
+  (!!((Yystate) == (-410)))
 
 #define YYTABLE_NINF -139
 
@@ -927,61 +930,62 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     174,  -395,  -395,   -64,    85,   -26,    14,   -31,   -16,  -395,
-      40,   196,   196,  -395,   109,   102,  -395,  -395,  -395,  -395,
-    -395,  -395,  -395,  -395,  -395,  -395,   148,    -3,    87,  -395,
-     -40,   121,   196,  -395,  -395,     1,    -5,   196,   196,   196,
-     196,   196,  -395,  -395,   716,    82,     2,  -395,   153,    63,
-    -395,  -395,  -395,    98,   152,    -3,    40,   141,  -395,    98,
-    -395,  -395,  -395,    12,    97,   116,   261,   116,   169,   126,
-     138,  -395,   176,  -395,  -395,   270,   274,  -395,  -395,  -395,
-     807,   139,  -395,   210,  -395,  -395,   154,  -395,  -395,   297,
-    -395,  -395,  -395,  -395,   172,  -395,  -395,   177,   231,   901,
-     313,   265,   192,  -395,  -395,   338,    23,  -395,  -395,   243,
-    -395,  -395,  -395,  -395,  -395,  1083,    -7,   196,   196,   214,
-     196,     1,   196,  -395,    98,   363,  -395,   205,   263,  -395,
-    -395,  -395,   255,  -395,   116,  -395,   196,   196,   625,  -395,
-    -395,   262,   196,  -395,  -395,  -395,   625,    33,   -29,  -395,
-     409,  -395,   165,   165,  1174,   411,  -395,   -14,    28,  -395,
-      13,   138,  1174,  -395,  -395,   196,  1174,  -395,  -395,  -395,
-    -395,  1174,    18,   274,  -395,   196,   398,    59,  -395,   417,
-    -395,    98,  -395,   202,  -395,   116,    98,    87,  -395,   196,
-      80,   196,   196,   196,  -395,   285,  -395,   211,  1241,   992,
-     214,   534,   422,   423,  -395,  -395,   312,   415,  1252,   219,
-      43,  1174,    61,  -395,  1174,  -395,   369,   292,  -395,  -395,
-    -395,  -395,  -395,  -395,   367,  -395,   216,   294,  -395,  -395,
-       7,   186,   267,  -395,   298,   186,     3,   372,  -395,  -395,
-      23,  -395,   347,  -395,  -395,   295,  1174,  -395,   351,   229,
-     196,  -395,  1174,  -395,   196,  -395,  -395,  -395,   303,   366,
-     368,   304,  -395,  -395,  -395,   232,  -395,  -395,  -395,  -395,
-    -395,    34,   196,   323,    80,   196,  -395,   188,  -395,  -395,
-       4,    65,   625,   625,   276,  -395,  -395,  -395,  -395,  -395,
-    -395,  -395,  -395,  1174,   311,  1174,    51,  -395,   234,   326,
-    1174,    71,  -395,   399,   351,  -395,  -395,  1174,   453,  -395,
-     160,   196,  -395,  -395,   370,  -395,   373,   374,   379,    13,
-    -395,   457,   462,   186,   430,   400,   431,   329,   380,  -395,
-     236,  -395,  1174,  -395,   351,  -395,   625,   333,   334,   196,
-    -395,   196,  -395,  -395,  -395,  -395,  -395,  -395,  -395,   196,
-    -395,  -395,  -395,   238,   454,   184,  -395,   336,   348,  -395,
-     391,   342,  1252,  -395,   403,   196,  -395,  -395,   188,  -395,
-    -395,   423,  -395,  -395,  -395,  1174,   345,   341,   901,  -395,
-     351,   401,  -395,  -395,  1252,   350,   351,  1174,  -395,    37,
-      35,  -395,  -395,  -395,  -395,  -395,    13,   267,   390,   395,
-    -395,  1174,   625,   396,  1174,  -395,   455,   108,  -395,   351,
-       8,   196,   196,   240,  -395,   242,  -395,   196,  -395,  -395,
-    -395,  -395,   354,    80,   461,   402,  -395,   625,  -395,  -395,
-     356,  -395,   346,   901,  -395,  1174,   245,  -395,  -395,  1252,
-     351,  -395,   495,  -395,   408,  -395,  -395,   358,   422,   464,
-     420,   358,  1174,  -395,  -395,  -395,   490,  -395,   249,   251,
-    -395,  -395,  -395,   196,  -395,  -395,   375,   468,  -395,    19,
-     196,  1174,   264,   351,  -395,   266,   371,   625,  1174,   504,
-     376,   377,  -395,   227,    46,   405,  -395,   269,   196,    -9,
-    -395,   381,   351,  -395,  -395,  -395,   422,   377,  -395,   196,
-    -395,   376,  -395,  1174,  -395,  -395,   421,   418,   407,   425,
-     515,   196,  -395,   277,  -395,  -395,   384,  -395,   496,  -395,
-    -395,    49,  -395,  -395,  -395,  -395,    56,   386,  -395,   196,
-     388,  -395,  -395,   466,   426,   467,  -395,   196,   279,   347,
-    -395,  -395,  -395,   281,   445,   404,  -395,   539,  -395,  -395
+     405,  -410,  -410,   -52,    43,   -13,    34,   -38,   116,  -410,
+      48,   231,   231,  -410,   115,   119,  -410,  -410,  -410,  -410,
+    -410,  -410,  -410,  -410,  -410,  -410,   166,    46,    49,  -410,
+     -36,   205,   231,  -410,  -410,    38,   132,   231,   231,   231,
+     231,   231,  -410,  -410,   652,    97,    72,  -410,   203,    95,
+    -410,  -410,  -410,   130,   175,    46,    48,   158,  -410,   130,
+    -410,  -410,  -410,    25,    61,   128,   270,   128,   184,   142,
+     149,  -410,   106,  -410,  -410,   309,   313,  -410,  -410,  -410,
+     743,   153,  -410,   218,  -410,  -410,   189,  -410,  -410,   331,
+    -410,  -410,  -410,  -410,   199,  -410,  -410,   207,   272,   837,
+     347,   305,   211,  -410,  -410,   328,    19,  -410,  -410,   262,
+    -410,  -410,  -410,  -410,  -410,  1019,     5,   231,   231,   220,
+     231,    38,   231,  -410,   130,   373,  -410,   148,   243,  -410,
+    -410,  -410,   244,  -410,   128,  -410,   231,   231,   561,  -410,
+    -410,   245,   231,  -410,  -410,  -410,   561,    52,    17,  -410,
+     400,  -410,   140,   140,  1110,   402,  -410,     1,    29,  -410,
+      20,   149,  1110,  -410,  -410,   231,  1110,  -410,  -410,  -410,
+    -410,  1110,    21,   313,  -410,   231,   323,   -64,  -410,   416,
+    -410,   130,  -410,   167,  -410,   128,   130,    49,  -410,   231,
+     152,   231,   231,   231,  -410,   269,  -410,   169,  1177,   928,
+     220,   470,   426,   433,  -410,  -410,   407,   428,  1188,   303,
+     173,    23,  1110,    64,  -410,  1110,  -410,   388,   306,  -410,
+    -410,  -410,  -410,  -410,  -410,   381,  -410,    66,   310,  -410,
+    -410,     8,   209,   215,  -410,   311,   209,     9,   385,  -410,
+    -410,    19,  -410,   360,  -410,  -410,   315,  1110,  -410,   301,
+     180,   231,  -410,  1110,  -410,   231,  -410,  -410,  -410,   320,
+     384,   386,   329,  -410,  -410,  -410,   196,  -410,  -410,  -410,
+    -410,  -410,    16,   231,   341,   152,   231,  -410,   174,  -410,
+    -410,     2,    81,   561,   561,   206,  -410,  -410,  -410,  -410,
+    -410,  -410,  -410,  -410,  1110,   334,  1110,    58,  -410,   210,
+     335,   349,  1110,    68,  -410,   420,   301,  -410,  -410,  1110,
+     476,  -410,   162,   231,  -410,  -410,   389,  -410,   390,   391,
+     406,    20,  -410,   486,   487,   209,   453,   421,   455,   353,
+     403,  -410,   223,  -410,  1110,  -410,   301,  -410,   561,   357,
+     358,   231,  -410,   231,  -410,  -410,  -410,  -410,  -410,  -410,
+    -410,   231,  -410,  -410,  -410,   225,   477,    86,  -410,   361,
+     366,  -410,   417,   364,  1188,  -410,   430,   231,  -410,  -410,
+     174,  -410,  -410,   433,  -410,  -410,  -410,  1110,   367,   284,
+     837,  -410,   301,   424,  -410,  -410,  1188,  1188,   374,   301,
+    1110,  -410,    33,   -16,  -410,  -410,  -410,  -410,  -410,    20,
+     215,   419,   422,  -410,  1110,   561,   429,  1110,  -410,   475,
+      18,  -410,   301,    31,   231,   231,   227,  -410,   233,  -410,
+     231,  -410,  -410,  -410,  -410,   382,   152,   489,   432,  -410,
+     561,  -410,  -410,   392,  -410,   289,   837,  -410,  1110,   238,
+    -410,  -410,   240,  1188,   301,  -410,   521,  -410,   437,  -410,
+    -410,   393,   426,   498,   456,   393,  1110,  -410,  -410,  -410,
+     522,  -410,   246,   252,  -410,  -410,  -410,   231,  -410,  -410,
+     399,   504,  -410,    32,   231,  1110,   254,   301,  -410,  -410,
+     257,   404,   561,  1110,   540,   412,   408,  -410,   274,    11,
+     442,  -410,   286,   231,   127,  -410,   410,   301,  -410,  -410,
+    -410,   426,   408,  -410,   231,  -410,   412,  -410,  1110,  -410,
+    -410,   460,   454,   447,   458,   549,   231,  -410,   290,  -410,
+    -410,   425,  -410,   528,  -410,  -410,    -6,  -410,  -410,  -410,
+    -410,    12,   431,  -410,   231,   434,  -410,  -410,   495,   462,
+     496,  -410,   231,   293,   360,  -410,  -410,  -410,   299,   473,
+     435,  -410,   563,  -410,  -410
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -989,93 +993,96 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   298,     0,     0,     0,     0,     0,     0,    18,
+       0,     6,   300,     0,     0,     0,     0,     0,     0,    18,
      123,     0,     0,     7,     0,     0,    15,     8,    10,    11,
       13,    14,     9,    17,    12,    16,     0,   112,   119,   121,
-       0,   296,     0,   290,   291,     0,     0,     0,     0,     0,
+       0,   298,     0,   292,   293,     0,     0,     0,     0,     0,
        0,     0,   124,   125,     0,     0,   114,   115,     0,   156,
        1,     3,     2,     0,     0,   112,   123,     0,   110,     0,
-       5,     4,   297,     0,     0,   103,     0,   103,     0,     0,
-     197,    25,     0,   256,   253,     0,   282,   126,    40,    29,
+       5,     4,   299,     0,     0,   103,     0,   103,     0,     0,
+     197,    25,     0,   256,   253,     0,   284,   126,    40,    29,
        0,     0,    30,    31,    34,    36,     0,    37,    39,     0,
       41,   252,    35,    38,     0,    32,    33,     0,     0,     0,
        0,     0,   127,   128,   232,   132,   218,   220,   222,   225,
-     228,   229,   230,   224,   223,     0,   268,     0,     0,     0,
+     228,   229,   230,   224,   223,     0,   270,     0,     0,     0,
        0,     0,     0,   111,     0,     0,   120,     0,     0,   100,
      102,   101,     0,    98,   103,    97,     0,     0,     0,   106,
      198,     0,     0,    94,   254,   255,     0,     0,   248,   245,
        0,    43,     0,   257,     0,     0,    44,     0,     0,   259,
-       0,   197,     0,   283,   284,     0,     0,   131,   286,   287,
-     285,     0,     0,     0,   221,     0,     0,   197,   108,     0,
-     116,     0,   117,     0,   288,   103,     0,   118,   113,     0,
+       0,   197,     0,   285,   286,     0,     0,   131,   288,   289,
+     287,     0,     0,     0,   221,     0,     0,   197,   108,     0,
+     116,     0,   117,     0,   290,   103,     0,   118,   113,     0,
        0,     0,     0,     0,    96,    66,    27,     0,     0,     0,
-       0,     0,   199,   201,   203,   205,     0,   223,     0,     0,
-       0,     0,   248,   242,     0,   246,     0,     0,   262,   263,
-     264,   261,   265,   260,     0,   258,     0,     0,   134,   231,
-       0,     0,   158,   147,   133,   152,   135,   160,   129,   130,
-     217,   219,   174,   226,   269,     0,     0,   233,   250,     0,
-       0,   105,     0,   157,     0,    99,    95,    19,     0,     0,
-       0,     0,    20,    21,    22,     0,    74,    76,    77,    78,
-      79,     0,     0,     0,    64,     0,    42,    56,   204,   212,
-       0,     0,     0,     0,     0,   272,   274,   275,   276,   277,
-     273,   278,   280,     0,     0,     0,     0,   266,     0,     0,
-       0,     0,   243,     0,   249,   241,    45,     0,     0,    46,
-     138,     0,   148,   154,   144,   139,   140,   142,     0,     0,
-     151,     0,     0,   150,     0,   162,     0,     0,   176,   234,
-       0,   235,     0,   107,   109,   289,     0,     0,     0,     0,
-     104,     0,    81,    84,    82,   294,   295,   293,   292,     0,
-      80,    85,   270,     0,   268,     0,    63,    65,    68,    28,
-       0,     0,     0,    47,     0,     0,    49,    55,    57,    26,
-     211,   200,   202,   279,   281,     0,     0,     0,     0,   213,
-     210,     0,   209,    93,     0,     0,   247,     0,   240,     0,
-       0,   153,   155,   145,   141,   143,     0,   159,     0,     0,
-     149,     0,     0,   164,     0,   227,     0,   178,   236,   251,
-       0,     0,     0,     0,    75,     0,    67,     0,    86,    87,
-      88,    89,    90,     0,     0,    70,    48,     0,    51,    50,
-       0,    54,     0,     0,   215,     0,     0,   208,   267,     0,
-     244,   237,     0,   238,     0,   136,   137,   161,   163,     0,
-     166,   175,     0,   181,   180,   173,     0,    61,     0,     0,
-      58,    83,   271,     0,    24,    62,     0,     0,    23,     0,
-       0,     0,     0,   206,   214,     0,     0,     0,     0,     0,
-     168,   177,   188,   191,     0,     0,    59,     0,     0,     0,
-      52,     0,   207,   216,    92,   239,   146,   165,   167,     0,
-     122,   169,   170,     0,   192,   193,   194,     0,     0,     0,
-       0,     0,    91,     0,    72,    73,     0,    53,     0,   171,
-     189,     0,   190,   182,   184,   183,     0,     0,    69,     0,
-       0,   195,   196,     0,     0,     0,   179,     0,     0,   174,
-     185,   187,   186,     0,     0,     0,    60,     0,   172,    71
+       0,     0,   199,   201,   203,   205,     0,   223,     0,    93,
+       0,     0,     0,   248,   242,     0,   246,     0,     0,   262,
+     263,   264,   261,   265,   260,     0,   258,     0,     0,   134,
+     231,     0,     0,   158,   147,   133,   152,   135,   160,   129,
+     130,   217,   219,   174,   226,   271,     0,     0,   233,   250,
+       0,     0,   105,     0,   157,     0,    99,    95,    19,     0,
+       0,     0,     0,    20,    21,    22,     0,    74,    76,    77,
+      78,    79,     0,     0,     0,    64,     0,    42,    56,   204,
+     212,     0,     0,     0,     0,     0,   274,   276,   277,   278,
+     279,   275,   280,   282,     0,     0,     0,     0,   266,     0,
+       0,     0,     0,     0,   243,     0,   249,   241,    45,     0,
+       0,    46,   138,     0,   148,   154,   144,   139,   140,   142,
+       0,     0,   151,     0,     0,   150,     0,   162,     0,     0,
+     176,   234,     0,   235,     0,   107,   109,   291,     0,     0,
+       0,     0,   104,     0,    81,    84,    82,   296,   297,   295,
+     294,     0,    80,    85,   272,     0,   270,     0,    63,    65,
+      68,    28,     0,     0,     0,    47,     0,     0,    49,    55,
+      57,    26,   211,   200,   202,   281,   283,     0,     0,     0,
+       0,   213,   210,     0,   209,   268,     0,     0,     0,   247,
+       0,   240,     0,     0,   153,   155,   145,   141,   143,     0,
+     159,     0,     0,   149,     0,     0,   164,     0,   227,     0,
+     178,   236,   251,     0,     0,     0,     0,    75,     0,    67,
+       0,    86,    87,    88,    89,    90,     0,     0,    70,    48,
+       0,    51,    50,     0,    54,     0,     0,   215,     0,     0,
+     208,   267,     0,     0,   244,   237,     0,   238,     0,   136,
+     137,   161,   163,     0,   166,   175,     0,   181,   180,   173,
+       0,    61,     0,     0,    58,    83,   273,     0,    24,    62,
+       0,     0,    23,     0,     0,     0,     0,   206,   214,   269,
+       0,     0,     0,     0,     0,   168,   177,   188,   191,     0,
+       0,    59,     0,     0,     0,    52,     0,   207,   216,    92,
+     239,   146,   165,   167,     0,   122,   169,   170,     0,   192,
+     193,   194,     0,     0,     0,     0,     0,    91,     0,    72,
+      73,     0,    53,     0,   171,   189,     0,   190,   182,   184,
+     183,     0,     0,    69,     0,     0,   195,   196,     0,     0,
+       0,   179,     0,     0,   174,   185,   187,   186,     0,     0,
+       0,    60,     0,   172,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -395,  -395,  -395,  -395,  -395,  -395,  -395,  -395,  -164,  -395,
-     349,   180,  -395,  -395,  -271,  -395,  -395,  -395,  -395,  -395,
-    -395,  -394,   209,  -395,  -395,  -395,  -395,  -395,  -395,  -395,
-    -395,    24,   -46,  -395,  -395,  -395,   301,  -395,   497,  -395,
-    -395,   435,   259,   433,   -28,   498,  -395,  -395,   397,  -395,
-     -90,  -395,  -395,  -207,   162,  -187,   -10,  -395,  -395,  -395,
-    -395,  -395,  -395,  -395,    60,    21,  -395,  -395,  -395,  -395,
-    -395,  -395,    84,    62,  -395,  -395,   -54,  -395,  -145,   282,
-     280,   382,   -35,   406,   412,   451,  -156,  -395,  -395,  -395,
-    -395,   355,  -395,   427,   359,  -232,  -203,   429,   129,  -128,
-    -395,  -395,  -395,  -395,  -395,  -136,    -4,  -395,  -395,  -395
+    -410,  -410,  -410,  -410,  -410,  -410,  -410,  -410,  -148,  -410,
+     376,   216,  -410,  -410,  -264,  -410,  -410,  -410,  -410,  -410,
+    -410,  -409,   236,  -410,  -410,  -410,  -410,  -410,  -410,  -410,
+    -410,    85,    -3,  -410,  -410,  -410,   325,  -410,   526,  -410,
+    -410,   467,   217,   464,   -43,   533,  -410,  -410,   436,  -410,
+    -114,  -410,  -410,  -186,   192,  -184,    -9,  -410,  -410,  -410,
+    -410,  -410,  -410,  -410,    89,    55,  -410,  -410,  -410,  -410,
+    -410,  -410,   109,    92,  -410,  -410,    79,  -410,  -142,   318,
+     322,   409,   -42,   441,   439,   501,  -154,  -410,  -410,  -410,
+    -410,   398,  -410,   469,   413,  -232,  -195,   449,  -324,  -410,
+    -129,  -410,  -410,  -410,  -410,  -410,  -141,    -4,  -410,  -410,
+    -410
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
       -1,    14,    15,    16,    17,    18,    19,    20,   196,   197,
-     100,   367,   368,   369,   262,   357,   358,   273,   425,   468,
-     516,   265,   266,   267,   268,   269,   270,   422,   464,    21,
+     100,   369,   370,   371,   263,   359,   360,   274,   428,   472,
+     521,   266,   267,   268,   269,   270,   271,   425,   468,    21,
       22,    65,   133,    23,    24,   177,   178,    25,    58,    26,
       46,    47,   157,    28,    29,    44,   101,   102,   103,   161,
-     104,   323,   318,   232,   233,   312,   313,   234,   325,   403,
-     450,   480,   500,   501,   502,   327,   328,   407,   455,   456,
-     510,   536,   481,   482,   506,   522,   139,   140,   202,   203,
+     104,   325,   320,   233,   234,   314,   315,   235,   327,   406,
+     454,   485,   505,   506,   507,   329,   330,   410,   459,   460,
+     515,   541,   486,   487,   511,   527,   139,   140,   202,   203,
      204,   205,   206,   106,   107,   108,   109,   110,   111,   112,
-     212,   213,   148,   149,   216,   249,   113,   224,   298,   114,
-     353,   295,   115,   166,   171,   183,   116,   351,    30,    31
+     213,   214,   148,   149,   217,   250,   113,   225,   299,   209,
+     114,   355,   296,   115,   166,   171,   183,   116,   353,    30,
+      31
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1083,256 +1090,215 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      36,   210,    48,   356,   235,   297,   209,    45,    49,   105,
-     207,    33,   282,    34,   330,    56,   282,    33,   207,    34,
-     175,   135,    33,   310,    34,   257,    56,   282,    63,   182,
-      56,   126,   214,    68,    69,    70,    71,    72,    33,   342,
-      34,   343,   321,   127,   143,   147,   168,   169,   320,   163,
-     164,   282,    37,   507,   163,   164,   280,    32,   163,   164,
-      67,   381,   344,   533,   158,   514,    66,    41,    42,   487,
-     231,   207,   128,   207,   235,   284,   285,   286,   287,   288,
-     289,   290,   291,   292,   293,    39,   163,   164,   194,    33,
-      40,    34,   163,   164,   513,    38,   508,   515,    43,   345,
-      60,   167,   129,    61,   442,   146,   534,   237,    48,    50,
-     279,   359,   397,   179,    49,   531,   184,    59,   186,   226,
-      64,    10,   214,   251,    62,   258,    64,   105,   346,   347,
-     322,   228,   195,   198,    57,   532,   400,   176,   184,   255,
-     231,   248,   294,   382,   352,   185,   436,   176,   118,   370,
-     259,   230,   465,   457,   207,   207,   236,   230,   256,   428,
-     348,   239,   242,   235,   490,   300,   281,   211,   243,   447,
-     170,   244,   451,   229,   509,     1,   301,     2,   349,   304,
-     443,   438,   441,   260,   535,   198,   119,   263,   264,   271,
-      33,   410,    34,   387,   138,   211,    53,   117,   360,    10,
-      33,   472,    34,   413,   379,   250,     3,   120,   207,   261,
-     229,   248,    10,   415,   453,   130,   131,   334,   218,   311,
-     454,   418,     4,     5,   419,   420,   236,    49,    54,   231,
-       6,    49,   314,   361,     7,   122,   297,   163,   164,   315,
-     235,   219,    51,   362,    52,   125,   179,   316,   163,   164,
-     335,   189,   190,   132,     8,   220,   221,   448,   377,    27,
-     380,   504,    10,    35,   136,   386,   134,   350,   354,   317,
-     137,   198,   389,   138,   207,   458,   459,   144,   222,     9,
-     363,   145,   469,   150,   505,    55,   434,   308,    10,   462,
-      10,   364,   151,   373,   374,   375,   365,   409,   152,   207,
-     421,   392,   153,   223,    11,   391,   231,    49,   141,   191,
-     192,    12,   121,   156,    13,   236,   154,   366,   159,    49,
-     142,   155,   284,   285,   286,   287,   288,   289,   290,   291,
-     292,   293,   496,   163,   164,   184,   160,   271,   162,   314,
-     432,   172,    33,   248,    34,   184,   315,   253,   254,   207,
-     435,  -138,   440,   376,   316,   471,   274,   275,   181,   163,
-     164,   430,   163,   164,   299,   254,   248,   163,   164,   248,
-     188,   165,   163,   164,   331,   332,   317,   340,   341,   383,
-     384,   408,   332,   416,   417,   460,   254,   461,   254,   294,
-     474,   332,   236,   538,   485,   254,   486,   254,   248,   193,
-     473,   543,    33,    73,    34,    74,   208,   184,   184,   493,
-     332,   494,   384,   354,   512,   341,   217,   483,   227,    75,
-      76,   245,   528,   341,   544,   254,   546,   254,   252,   272,
-     282,   305,   283,    78,    79,   296,   492,   306,   307,   309,
-     329,    80,    81,   483,   319,   324,   326,   336,   339,    82,
-      83,   337,    84,   338,   355,   378,   246,    85,   385,   271,
-     390,   388,    86,   396,   398,    87,   491,   393,   483,   399,
-     394,   395,   401,   404,   405,   402,   406,   411,   412,    88,
-      89,   175,   423,   426,   271,   424,   427,    90,   429,   433,
-      91,   445,   449,   437,   439,   518,   446,   452,   463,   466,
-     470,   467,   476,   477,   332,    92,   478,   527,   479,   484,
-     489,   498,   499,   511,   521,    93,   495,   524,    94,   488,
-     523,    95,    96,   503,   526,   184,   517,   525,   529,   530,
-     537,    97,   539,   184,   540,   542,   541,    98,    33,    73,
-      34,    74,    99,   247,   199,   547,   549,   277,   431,   548,
-     414,   333,   123,   180,   124,    75,    76,   187,   444,   238,
-     545,   519,   497,   372,   371,   520,   174,   302,   475,    78,
-      79,   303,   240,     0,     0,   215,     0,    80,    81,     0,
-       0,   278,   225,   241,     0,    82,    83,     0,    84,     0,
-       0,     0,     0,    85,     0,     0,     0,   200,    86,     0,
-       0,    87,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,    88,    89,     0,     0,     0,
-       0,     0,     0,    90,     0,     0,    91,     0,     0,    33,
-      73,    34,    74,     0,     0,   199,     0,     0,     0,     0,
-       0,    92,     0,     0,     0,     0,    75,    76,    10,     0,
-       0,    93,     0,     0,    94,     0,     0,    95,    96,     0,
-      78,    79,     0,     0,     0,     0,     0,    97,    80,    81,
-       0,     0,     0,    98,     0,     0,    82,    83,   201,    84,
-       0,     0,     0,     0,    85,     0,     0,     0,   200,    86,
-       0,     0,    87,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,    88,    89,     0,     0,
-       0,     0,     0,     0,    90,     0,     0,    91,     0,     0,
-      33,    73,    34,    74,     0,     0,     0,     0,     0,     0,
-       0,     0,    92,     0,     0,     0,     0,    75,    76,    77,
-       0,     0,    93,     0,     0,    94,     0,     0,    95,    96,
-       0,    78,    79,     0,     0,     0,     0,     0,    97,    80,
-      81,     0,     0,     0,    98,     0,     0,    82,    83,   201,
-      84,     0,     0,     0,     0,    85,     0,     0,     0,     0,
-      86,     0,     0,    87,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    88,    89,     0,
-       0,     0,     0,     0,     0,    90,     0,     0,    91,     0,
-       0,    33,    73,    34,    74,     0,     0,     0,     0,     0,
-       0,     0,     0,    92,     0,     0,     0,     0,    75,    76,
-       0,     0,     0,    93,     0,     0,    94,     0,     0,    95,
-      96,     0,    78,    79,     0,     0,     0,     0,     0,    97,
-      80,    81,     0,     0,     0,    98,     0,     0,    82,    83,
-      99,    84,     0,     0,     0,     0,    85,     0,     0,     0,
-       0,    86,     0,     0,    87,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    88,    89,
-       0,     0,     0,     0,     0,     0,    90,     0,     0,    91,
-       0,     0,     0,     0,     0,    33,    73,    34,    74,     0,
-       0,     0,     0,     0,    92,     0,     0,     0,     0,     0,
-       0,     0,    75,    76,    93,     0,     0,    94,     0,     0,
-      95,    96,     0,     0,     0,     0,    78,    79,     0,     0,
-      97,   146,     0,     0,    80,    81,    98,     0,     0,     0,
-       0,    99,    82,    83,     0,    84,     0,     0,     0,     0,
-      85,     0,     0,     0,     0,    86,     0,     0,    87,     0,
+      36,   210,   105,    48,   211,   182,   236,    45,    49,   207,
+     283,   358,    33,   298,    34,   332,   126,   207,   512,   538,
+      33,   344,    34,   345,    33,    33,    34,    34,    63,   143,
+      56,   283,   175,    68,    69,    70,    71,    72,   147,   283,
+     283,   258,   168,   169,   346,   312,   232,    33,   323,    34,
+     163,   164,   322,   446,   163,   164,   127,   158,   492,   281,
+     536,   513,   539,   442,   135,    37,   129,    56,   383,    32,
+     207,   138,   207,   163,   164,    56,    42,   236,   215,    59,
+     537,   347,   251,    40,   518,   128,   280,   163,   164,   163,
+     164,   285,   286,   287,   288,   289,   290,   291,   292,   293,
+     294,   167,   163,   164,    60,    39,    43,    61,    38,    48,
+     348,   349,   227,   179,    49,    50,   184,   232,   186,   480,
+     105,    67,    10,   421,   457,   215,   422,   423,   361,   447,
+     458,   194,   195,   198,   249,   400,   324,   310,   184,   514,
+     540,   403,   350,   257,   354,   302,   229,   372,   439,   176,
+     384,   146,   231,   176,   207,   207,   237,    10,   295,   282,
+     351,   240,   469,    64,   231,   243,   170,   236,   244,   431,
+     303,   245,   451,   306,   230,   455,   461,   495,   445,   130,
+     131,   381,   256,    57,   362,   198,   212,   264,   265,   272,
+     390,   441,   298,   219,   189,   190,   413,   259,   212,    41,
+     416,   519,   424,    66,   476,   249,   185,   232,    62,   207,
+     418,   336,   117,    33,    53,    34,   220,    27,   118,   363,
+      10,    35,   260,   375,   376,   377,   230,   237,    49,   364,
+     221,   222,    49,   520,   316,    33,   119,    34,   141,   120,
+     238,   317,   313,    55,    10,   236,    54,   179,   298,   318,
+     142,   337,   379,   223,   382,   261,   252,    64,   122,    51,
+     389,    52,   125,   452,   437,   132,   365,   392,   352,   356,
+     121,   319,   198,   462,   463,   134,   207,   366,   224,   136,
+      10,   262,   367,   378,   138,   232,   137,   316,   473,   191,
+     192,   466,   412,   438,   317,   163,   164,   150,   475,  -138,
+     151,   207,   318,   368,   395,   163,   164,   394,   509,    49,
+     163,   164,   254,   255,   275,   276,   144,   237,   301,   255,
+     145,    49,   163,   164,   319,   333,   334,    33,    73,    34,
+      74,   510,    33,   152,    34,   435,   153,   184,   249,   272,
+     501,   342,   343,   154,    75,    76,   246,   184,   444,   163,
+     164,   155,   159,   207,   156,   385,   386,   162,    78,    79,
+     172,   165,   249,   433,   181,   249,    80,    81,   411,   334,
+     419,   420,   464,   255,    82,    83,   160,    84,   465,   255,
+     188,   247,    85,   478,   334,   479,   386,    86,   193,   208,
+      87,   490,   255,   543,   249,   237,   477,   491,   255,   498,
+     334,   548,   499,   386,    88,    89,     1,   218,     2,   228,
+     184,   184,    90,   273,   488,    91,   356,   285,   286,   287,
+     288,   289,   290,   291,   292,   293,   294,   253,   163,   164,
+      92,   517,   343,   497,   283,   533,   343,     3,   549,   255,
+      93,   488,   284,    94,   551,   255,    95,    96,   297,   300,
+     307,   308,   309,     4,     5,   311,    97,   321,   326,   328,
+     331,     6,    98,   272,   338,     7,   488,    99,   248,   339,
+     496,   340,   357,   341,    33,    73,    34,    74,   380,   387,
+     199,   388,   391,   393,   295,     8,   396,   397,   398,   272,
+     399,    75,    76,   401,   402,   404,   405,   407,   408,   409,
+     523,   414,   415,   427,   175,    78,    79,   426,   430,   429,
+       9,   436,   532,    80,    81,   432,   440,   456,   443,    10,
+     449,    82,    83,   450,    84,   453,   467,   470,   481,    85,
+     184,   471,   482,   200,    86,    11,   474,    87,   184,   334,
+     483,   489,    12,   493,   484,    13,   494,   503,   504,   500,
+     516,    88,    89,   526,   508,   522,   528,   529,   531,    90,
+     530,   535,    91,   545,   547,    33,    73,    34,    74,   534,
+     554,   199,   546,   552,   278,   542,   335,    92,   544,   417,
+     553,   123,    75,    76,    10,   180,   434,    93,   187,   124,
+      94,   448,   502,    95,    96,   524,    78,    79,   239,   550,
+     525,   373,   226,    97,    80,    81,   374,   241,   279,    98,
+     242,   304,    82,    83,   201,    84,   174,   216,     0,     0,
+      85,     0,     0,     0,   200,    86,   305,     0,    87,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,    88,    89,     0,     0,     0,     0,     0,     0,
       90,     0,     0,    91,     0,     0,    33,    73,    34,    74,
        0,     0,     0,     0,     0,     0,     0,     0,    92,     0,
-       0,     0,     0,    75,    76,    10,     0,     0,    93,     0,
+       0,     0,     0,    75,    76,    77,     0,     0,    93,     0,
        0,    94,     0,     0,    95,    96,     0,    78,    79,     0,
        0,     0,     0,     0,    97,    80,    81,     0,     0,     0,
-      98,     0,     0,    82,    83,    99,    84,     0,     0,     0,
-       0,    85,     0,     0,     0,   200,    86,     0,     0,    87,
+      98,     0,     0,    82,    83,   201,    84,     0,     0,     0,
+       0,    85,     0,     0,     0,     0,    86,     0,     0,    87,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,     0,    88,    89,     0,     0,     0,     0,     0,
        0,    90,     0,     0,    91,     0,     0,    33,    73,    34,
       74,     0,     0,     0,     0,     0,     0,     0,     0,    92,
-       0,     0,     0,     0,    75,   173,     0,     0,     0,    93,
+       0,     0,     0,     0,    75,    76,     0,     0,     0,    93,
        0,     0,    94,     0,     0,    95,    96,     0,    78,    79,
        0,     0,     0,     0,     0,    97,    80,    81,     0,     0,
-       0,    98,     0,     0,    82,    83,   201,    84,     0,     0,
+       0,    98,     0,     0,    82,    83,    99,    84,     0,     0,
        0,     0,    85,     0,     0,     0,     0,    86,     0,     0,
       87,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,     0,     0,    88,    89,     0,     0,     0,     0,
-       0,     0,    90,     0,     0,    91,     0,     0,    33,    73,
-      34,    74,     0,     0,     0,     0,     0,     0,     0,     0,
-      92,     0,     0,     0,     0,    75,    76,     0,     0,     0,
-      93,     0,     0,    94,     0,     0,    95,    96,     0,    78,
-      79,     0,     0,     0,     0,     0,    97,    80,    81,     0,
-       0,     0,    98,     0,     0,    82,    83,    99,    84,     0,
-       0,     0,     0,    85,     0,     0,     0,     0,    86,     0,
-       0,    87,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,    88,    89,    73,     0,    74,
-       0,     0,     0,    90,     0,     0,    91,     0,     0,     0,
-       0,     0,     0,    75,   173,     0,    78,    79,     0,     0,
-       0,    92,     0,     0,     0,    81,     0,    78,    79,     0,
-       0,    93,    82,    83,    94,    84,    81,    95,    96,     0,
-      85,     0,     0,    82,    83,     0,    84,    97,    87,     0,
-       0,    85,     0,    98,     0,     0,     0,     0,    99,    87,
-       0,     0,    88,   276,     0,     0,     0,     0,     0,     0,
-      90,     0,     0,    88,    89,     0,     0,     0,     0,     0,
-       0,    90,     0,     0,    91,     0,     0,     0,    92,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    93,    92,
-       0,     0,     0,     0,    95,    96,     0,     0,     0,    93,
-       0,     0,     0,     0,    97,    95,    96,     0,     0,     0,
-      98,     0,     0,     0,     0,    97,     0,     0,     0,     0,
-       0,    98
+       0,     0,    90,     0,     0,    91,     0,     0,     0,     0,
+       0,    33,    73,    34,    74,     0,     0,     0,     0,     0,
+      92,     0,     0,     0,     0,     0,     0,     0,    75,    76,
+      93,     0,     0,    94,     0,     0,    95,    96,     0,     0,
+       0,     0,    78,    79,     0,     0,    97,   146,     0,     0,
+      80,    81,    98,     0,     0,     0,     0,    99,    82,    83,
+       0,    84,     0,     0,     0,     0,    85,     0,     0,     0,
+       0,    86,     0,     0,    87,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,    88,    89,
+       0,     0,     0,     0,     0,     0,    90,     0,     0,    91,
+       0,     0,    33,    73,    34,    74,     0,     0,     0,     0,
+       0,     0,     0,     0,    92,     0,     0,     0,     0,    75,
+      76,    10,     0,     0,    93,     0,     0,    94,     0,     0,
+      95,    96,     0,    78,    79,     0,     0,     0,     0,     0,
+      97,    80,    81,     0,     0,     0,    98,     0,     0,    82,
+      83,    99,    84,     0,     0,     0,     0,    85,     0,     0,
+       0,   200,    86,     0,     0,    87,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,    88,
+      89,     0,     0,     0,     0,     0,     0,    90,     0,     0,
+      91,     0,     0,    33,    73,    34,    74,     0,     0,     0,
+       0,     0,     0,     0,     0,    92,     0,     0,     0,     0,
+      75,   173,     0,     0,     0,    93,     0,     0,    94,     0,
+       0,    95,    96,     0,    78,    79,     0,     0,     0,     0,
+       0,    97,    80,    81,     0,     0,     0,    98,     0,     0,
+      82,    83,   201,    84,     0,     0,     0,     0,    85,     0,
+       0,     0,     0,    86,     0,     0,    87,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+      88,    89,     0,     0,     0,     0,     0,     0,    90,     0,
+       0,    91,     0,     0,    33,    73,    34,    74,     0,     0,
+       0,     0,     0,     0,     0,     0,    92,     0,     0,     0,
+       0,    75,    76,     0,     0,     0,    93,     0,     0,    94,
+       0,     0,    95,    96,     0,    78,    79,     0,     0,     0,
+       0,     0,    97,    80,    81,     0,     0,     0,    98,     0,
+       0,    82,    83,    99,    84,     0,     0,     0,     0,    85,
+       0,     0,     0,     0,    86,     0,     0,    87,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,    88,    89,    73,     0,    74,     0,     0,     0,    90,
+       0,     0,    91,     0,     0,     0,     0,     0,     0,    75,
+     173,     0,    78,    79,     0,     0,     0,    92,     0,     0,
+       0,    81,     0,    78,    79,     0,     0,    93,    82,    83,
+      94,    84,    81,    95,    96,     0,    85,     0,     0,    82,
+      83,     0,    84,    97,    87,     0,     0,    85,     0,    98,
+       0,     0,     0,     0,    99,    87,     0,     0,    88,   277,
+       0,     0,     0,     0,     0,     0,    90,     0,     0,    88,
+      89,     0,     0,     0,     0,     0,     0,    90,     0,     0,
+      91,     0,     0,     0,    92,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,    93,    92,     0,     0,     0,     0,
+      95,    96,     0,     0,     0,    93,     0,     0,     0,     0,
+      97,    95,    96,     0,     0,     0,    98,     0,     0,     0,
+       0,    97,     0,     0,     0,     0,     0,    98
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   146,    12,   274,   160,   208,   142,    11,    12,    44,
-     138,     4,     8,     6,   246,    29,     8,     4,   146,     6,
-      27,    67,     4,   230,     6,   189,    29,     8,    32,   119,
-      29,    59,    61,    37,    38,    39,    40,    41,     4,     5,
-       6,     7,    39,    31,    72,    80,    23,    24,   235,    21,
-      22,     8,    78,     7,    21,    22,   201,   121,    21,    22,
-      36,    10,    28,     7,    99,    74,    71,    83,    28,   463,
-     160,   199,    60,   201,   230,    10,    11,    12,    13,    14,
-      15,    16,    17,    18,    19,    71,    21,    22,   134,     4,
-     121,     6,    21,    22,   488,   121,    50,   106,    58,    65,
-     140,   105,     5,   143,    69,   134,    50,   161,   118,     0,
-     200,   275,   319,   117,   118,    66,   120,    30,   122,   154,
-     125,   114,    61,   177,     3,    45,   125,   162,    94,    95,
-     127,   145,   136,   137,   137,    86,   323,   144,   142,   185,
-     230,   176,    77,    92,   272,   121,   378,   144,   146,   145,
-      70,   144,   423,   145,   282,   283,   160,   144,   186,   362,
-     126,   165,   144,   319,   145,   122,   201,   134,   172,   401,
-     147,   175,   404,   145,   128,     1,   211,     3,   144,   214,
-     145,   384,   145,   103,   128,   189,    33,   191,   192,   193,
-       4,   336,     6,   122,   135,   134,    48,   115,    10,   114,
-       4,   433,     6,   339,   294,   146,    32,   144,   336,   129,
-     145,   246,   114,   349,   106,   118,   119,   252,    53,    33,
-     112,    37,    48,    49,    40,    41,   230,   231,    80,   319,
-      56,   235,    72,    45,    60,    83,   439,    21,    22,    79,
-     396,    76,   140,    55,   142,   104,   250,    87,    21,    22,
-     254,    46,    47,   137,    80,    90,    91,   402,   293,     0,
-     295,    34,   114,     4,    95,   300,     5,   271,   272,   109,
-     144,   275,   307,   135,   402,   411,   412,     7,   113,   105,
-      92,     7,   427,   144,    57,    26,   376,    71,   114,   417,
-     114,   103,    82,    17,    18,    19,   108,   332,   144,   427,
-     116,   311,     5,   138,   130,   145,   396,   311,   132,    46,
-      47,   137,    53,    82,   140,   319,   144,   129,     5,   323,
-     144,   144,    10,    11,    12,    13,    14,    15,    16,    17,
-      18,    19,   477,    21,    22,   339,    71,   341,   146,    72,
-     375,    98,     4,   378,     6,   349,    79,   145,   146,   477,
-       9,    84,   387,    77,    87,     9,   145,   146,   144,    21,
-      22,   365,    21,    22,   145,   146,   401,    21,    22,   404,
-       7,    33,    21,    22,   145,   146,   109,   145,   146,   145,
-     146,   145,   146,   145,   146,   145,   146,   145,   146,    77,
-     145,   146,   396,   529,   145,   146,   145,   146,   433,   144,
-     435,   537,     4,     5,     6,     7,   144,   411,   412,   145,
-     146,   145,   146,   417,   145,   146,     7,   452,     7,    21,
-      22,    23,   145,   146,   145,   146,   145,   146,    11,   144,
-       8,    62,     9,    35,    36,    20,   471,   145,    71,   145,
-     145,    43,    44,   478,   146,    73,    99,   144,   144,    51,
-      52,    85,    54,    85,   131,   144,    58,    59,   132,   463,
-       7,    62,    64,    84,     7,    67,   470,    97,   503,     7,
-      97,    97,    42,    42,   145,    75,    96,   144,   144,    81,
-      82,    27,   146,    92,   488,   137,   144,    89,    85,   144,
-      92,   101,    96,    92,   144,   499,   101,    42,   144,    38,
-     144,    99,     7,    95,   146,   107,    42,   511,    88,    19,
-      42,     7,   136,   108,    93,   117,   145,   110,   120,   144,
-     102,   123,   124,   146,     9,   529,   145,   102,   144,    33,
-     144,   133,   144,   537,    68,    68,   110,   139,     4,     5,
-       6,     7,   144,   145,    10,   100,     7,   198,   368,   145,
-     341,   250,    55,   118,    56,    21,    22,   124,   396,   162,
-     539,   501,   478,   283,   282,   503,   115,   212,   439,    35,
-      36,   212,   166,    -1,    -1,   148,    -1,    43,    44,    -1,
-      -1,   199,   153,   171,    -1,    51,    52,    -1,    54,    -1,
-      -1,    -1,    -1,    59,    -1,    -1,    -1,    63,    64,    -1,
-      -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    81,    82,    -1,    -1,    -1,
-      -1,    -1,    -1,    89,    -1,    -1,    92,    -1,    -1,     4,
-       5,     6,     7,    -1,    -1,    10,    -1,    -1,    -1,    -1,
-      -1,   107,    -1,    -1,    -1,    -1,    21,    22,   114,    -1,
-      -1,   117,    -1,    -1,   120,    -1,    -1,   123,   124,    -1,
-      35,    36,    -1,    -1,    -1,    -1,    -1,   133,    43,    44,
-      -1,    -1,    -1,   139,    -1,    -1,    51,    52,   144,    54,
-      -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,    63,    64,
-      -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    81,    82,    -1,    -1,
-      -1,    -1,    -1,    -1,    89,    -1,    -1,    92,    -1,    -1,
-       4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,   107,    -1,    -1,    -1,    -1,    21,    22,    23,
-      -1,    -1,   117,    -1,    -1,   120,    -1,    -1,   123,   124,
-      -1,    35,    36,    -1,    -1,    -1,    -1,    -1,   133,    43,
-      44,    -1,    -1,    -1,   139,    -1,    -1,    51,    52,   144,
-      54,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,    -1,
-      64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,    -1,
-      -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,    -1,
-      -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    21,    22,
-      -1,    -1,    -1,   117,    -1,    -1,   120,    -1,    -1,   123,
-     124,    -1,    35,    36,    -1,    -1,    -1,    -1,    -1,   133,
-      43,    44,    -1,    -1,    -1,   139,    -1,    -1,    51,    52,
-     144,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,
-      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,    -1,    -1,    -1,    -1,     4,     5,     6,     7,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    21,    22,   117,    -1,    -1,   120,    -1,    -1,
-     123,   124,    -1,    -1,    -1,    -1,    35,    36,    -1,    -1,
-     133,   134,    -1,    -1,    43,    44,   139,    -1,    -1,    -1,
-      -1,   144,    51,    52,    -1,    54,    -1,    -1,    -1,    -1,
-      59,    -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,
+       4,   142,    44,    12,   146,   119,   160,    11,    12,   138,
+       8,   275,     4,   208,     6,   247,    59,   146,     7,     7,
+       4,     5,     6,     7,     4,     4,     6,     6,    32,    72,
+      29,     8,    27,    37,    38,    39,    40,    41,    80,     8,
+       8,   189,    23,    24,    28,   231,   160,     4,    39,     6,
+      21,    22,   236,    69,    21,    22,    31,    99,   467,   201,
+      66,    50,    50,   387,    67,    78,     5,    29,    10,   121,
+     199,   135,   201,    21,    22,    29,    28,   231,    61,    30,
+      86,    65,   146,   121,   493,    60,   200,    21,    22,    21,
+      22,    10,    11,    12,    13,    14,    15,    16,    17,    18,
+      19,   105,    21,    22,   140,    71,    58,   143,   121,   118,
+      94,    95,   154,   117,   118,     0,   120,   231,   122,   443,
+     162,    36,   114,    37,   106,    61,    40,    41,   276,   145,
+     112,   134,   136,   137,   176,   321,   127,    71,   142,   128,
+     128,   325,   126,   186,   273,   122,   145,   145,   380,   144,
+      92,   134,   144,   144,   283,   284,   160,   114,    77,   201,
+     144,   165,   426,   125,   144,   144,   147,   321,   172,   364,
+     212,   175,   404,   215,   145,   407,   145,   145,   145,   118,
+     119,   295,   185,   137,    10,   189,   134,   191,   192,   193,
+     122,   386,   387,    53,    46,    47,   338,    45,   134,    83,
+     341,    74,   116,    71,   436,   247,   121,   321,     3,   338,
+     351,   253,   115,     4,    48,     6,    76,     0,   146,    45,
+     114,     4,    70,    17,    18,    19,   145,   231,   232,    55,
+      90,    91,   236,   106,    72,     4,    33,     6,   132,   144,
+     161,    79,    33,    26,   114,   399,    80,   251,   443,    87,
+     144,   255,   294,   113,   296,   103,   177,   125,    83,   140,
+     302,   142,   104,   405,   378,   137,    92,   309,   272,   273,
+      53,   109,   276,   414,   415,     5,   405,   103,   138,    95,
+     114,   129,   108,    77,   135,   399,   144,    72,   430,    46,
+      47,   420,   334,     9,    79,    21,    22,   144,     9,    84,
+      82,   430,    87,   129,   313,    21,    22,   145,    34,   313,
+      21,    22,   145,   146,   145,   146,     7,   321,   145,   146,
+       7,   325,    21,    22,   109,   145,   146,     4,     5,     6,
+       7,    57,     4,   144,     6,   377,     5,   341,   380,   343,
+     482,   145,   146,   144,    21,    22,    23,   351,   390,    21,
+      22,   144,     5,   482,    82,   145,   146,   146,    35,    36,
+      98,    33,   404,   367,   144,   407,    43,    44,   145,   146,
+     145,   146,   145,   146,    51,    52,    71,    54,   145,   146,
+       7,    58,    59,   145,   146,   145,   146,    64,   144,   144,
+      67,   145,   146,   534,   436,   399,   438,   145,   146,   145,
+     146,   542,   145,   146,    81,    82,     1,     7,     3,     7,
+     414,   415,    89,   144,   456,    92,   420,    10,    11,    12,
+      13,    14,    15,    16,    17,    18,    19,    11,    21,    22,
+     107,   145,   146,   475,     8,   145,   146,    32,   145,   146,
+     117,   483,     9,   120,   145,   146,   123,   124,    20,   146,
+      62,   145,    71,    48,    49,   145,   133,   146,    73,    99,
+     145,    56,   139,   467,   144,    60,   508,   144,   145,    85,
+     474,    85,   131,   144,     4,     5,     6,     7,   144,   144,
+      10,   132,    62,     7,    77,    80,    97,    97,    97,   493,
+      84,    21,    22,     7,     7,    42,    75,    42,   145,    96,
+     504,   144,   144,   137,    27,    35,    36,   146,   144,    92,
+     105,   144,   516,    43,    44,    85,    92,    42,   144,   114,
+     101,    51,    52,   101,    54,    96,   144,    38,     7,    59,
+     534,    99,    95,    63,    64,   130,   144,    67,   542,   146,
+      42,    19,   137,   144,    88,   140,    42,     7,   136,   145,
+     108,    81,    82,    93,   146,   145,   102,   110,     9,    89,
+     102,    33,    92,    68,    68,     4,     5,     6,     7,   144,
+       7,    10,   110,   100,   198,   144,   251,   107,   144,   343,
+     145,    55,    21,    22,   114,   118,   370,   117,   124,    56,
+     120,   399,   483,   123,   124,   506,    35,    36,   162,   544,
+     508,   283,   153,   133,    43,    44,   284,   166,   199,   139,
+     171,   213,    51,    52,   144,    54,   115,   148,    -1,    -1,
+      59,    -1,    -1,    -1,    63,    64,   213,    -1,    67,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,
       89,    -1,    -1,    92,    -1,    -1,     4,     5,     6,     7,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,
-      -1,    -1,    -1,    21,    22,   114,    -1,    -1,   117,    -1,
+      -1,    -1,    -1,    21,    22,    23,    -1,    -1,   117,    -1,
       -1,   120,    -1,    -1,   123,   124,    -1,    35,    36,    -1,
       -1,    -1,    -1,    -1,   133,    43,    44,    -1,    -1,    -1,
      139,    -1,    -1,    51,    52,   144,    54,    -1,    -1,    -1,
-      -1,    59,    -1,    -1,    -1,    63,    64,    -1,    -1,    67,
+      -1,    59,    -1,    -1,    -1,    -1,    64,    -1,    -1,    67,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,
       -1,    89,    -1,    -1,    92,    -1,    -1,     4,     5,     6,
@@ -1344,29 +1310,56 @@ static const yytype_int16 yycheck[] =
       -1,    -1,    59,    -1,    -1,    -1,    -1,    64,    -1,    -1,
       67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,
-      -1,    -1,    89,    -1,    -1,    92,    -1,    -1,     4,     5,
-       6,     7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-     107,    -1,    -1,    -1,    -1,    21,    22,    -1,    -1,    -1,
-     117,    -1,    -1,   120,    -1,    -1,   123,   124,    -1,    35,
-      36,    -1,    -1,    -1,    -1,    -1,   133,    43,    44,    -1,
-      -1,    -1,   139,    -1,    -1,    51,    52,   144,    54,    -1,
-      -1,    -1,    -1,    59,    -1,    -1,    -1,    -1,    64,    -1,
-      -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    81,    82,     5,    -1,     7,
-      -1,    -1,    -1,    89,    -1,    -1,    92,    -1,    -1,    -1,
-      -1,    -1,    -1,    21,    22,    -1,    35,    36,    -1,    -1,
-      -1,   107,    -1,    -1,    -1,    44,    -1,    35,    36,    -1,
-      -1,   117,    51,    52,   120,    54,    44,   123,   124,    -1,
-      59,    -1,    -1,    51,    52,    -1,    54,   133,    67,    -1,
-      -1,    59,    -1,   139,    -1,    -1,    -1,    -1,   144,    67,
-      -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,
-      89,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,
-      -1,    89,    -1,    -1,    92,    -1,    -1,    -1,   107,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   117,   107,
-      -1,    -1,    -1,    -1,   123,   124,    -1,    -1,    -1,   117,
-      -1,    -1,    -1,    -1,   133,   123,   124,    -1,    -1,    -1,
-     139,    -1,    -1,    -1,    -1,   133,    -1,    -1,    -1,    -1,
-      -1,   139
+      -1,    -1,    89,    -1,    -1,    92,    -1,    -1,    -1,    -1,
+      -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
+     107,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    21,    22,
+     117,    -1,    -1,   120,    -1,    -1,   123,   124,    -1,    -1,
+      -1,    -1,    35,    36,    -1,    -1,   133,   134,    -1,    -1,
+      43,    44,   139,    -1,    -1,    -1,    -1,   144,    51,    52,
+      -1,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,
+      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
+      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
+      -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    21,
+      22,   114,    -1,    -1,   117,    -1,    -1,   120,    -1,    -1,
+     123,   124,    -1,    35,    36,    -1,    -1,    -1,    -1,    -1,
+     133,    43,    44,    -1,    -1,    -1,   139,    -1,    -1,    51,
+      52,   144,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,
+      -1,    63,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,
+      82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,
+      92,    -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,
+      21,    22,    -1,    -1,    -1,   117,    -1,    -1,   120,    -1,
+      -1,   123,   124,    -1,    35,    36,    -1,    -1,    -1,    -1,
+      -1,   133,    43,    44,    -1,    -1,    -1,   139,    -1,    -1,
+      51,    52,   144,    54,    -1,    -1,    -1,    -1,    59,    -1,
+      -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,
+      -1,    92,    -1,    -1,     4,     5,     6,     7,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,
+      -1,    21,    22,    -1,    -1,    -1,   117,    -1,    -1,   120,
+      -1,    -1,   123,   124,    -1,    35,    36,    -1,    -1,    -1,
+      -1,    -1,   133,    43,    44,    -1,    -1,    -1,   139,    -1,
+      -1,    51,    52,   144,    54,    -1,    -1,    -1,    -1,    59,
+      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    81,    82,     5,    -1,     7,    -1,    -1,    -1,    89,
+      -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    21,
+      22,    -1,    35,    36,    -1,    -1,    -1,   107,    -1,    -1,
+      -1,    44,    -1,    35,    36,    -1,    -1,   117,    51,    52,
+     120,    54,    44,   123,   124,    -1,    59,    -1,    -1,    51,
+      52,    -1,    54,   133,    67,    -1,    -1,    59,    -1,   139,
+      -1,    -1,    -1,    -1,   144,    67,    -1,    -1,    81,    82,
+      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    81,
+      82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,
+      92,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,   117,   107,    -1,    -1,    -1,    -1,
+     123,   124,    -1,    -1,    -1,   117,    -1,    -1,    -1,    -1,
+     133,   123,   124,    -1,    -1,    -1,   139,    -1,    -1,    -1,
+      -1,   133,    -1,    -1,    -1,    -1,    -1,   139
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
@@ -1376,58 +1369,59 @@ static const yytype_uint16 yystos[] =
        0,     1,     3,    32,    48,    49,    56,    60,    80,   105,
      114,   130,   137,   140,   149,   150,   151,   152,   153,   154,
      155,   177,   178,   181,   182,   185,   187,   190,   191,   192,
-     256,   257,   121,     4,     6,   190,   254,    78,   121,    71,
-     121,    83,    28,    58,   193,   254,   188,   189,   204,   254,
+     257,   258,   121,     4,     6,   190,   255,    78,   121,    71,
+     121,    83,    28,    58,   193,   255,   188,   189,   204,   255,
        0,   140,   142,    48,    80,   190,    29,   137,   186,    30,
-     140,   143,     3,   254,   125,   179,    71,   179,   254,   254,
-     254,   254,   254,     5,     7,    21,    22,    23,    35,    36,
+     140,   143,     3,   255,   125,   179,    71,   179,   255,   255,
+     255,   255,   255,     5,     7,    21,    22,    23,    35,    36,
       43,    44,    51,    52,    54,    59,    64,    67,    81,    82,
       89,    92,   107,   117,   120,   123,   124,   133,   139,   144,
      158,   194,   195,   196,   198,   230,   231,   232,   233,   234,
-     235,   236,   237,   244,   247,   250,   254,   115,   146,    33,
+     235,   236,   237,   244,   248,   251,   255,   115,   146,    33,
      144,   190,    83,   186,   193,   104,   192,    31,    60,     5,
      118,   119,   137,   180,     5,   180,    95,   144,   135,   224,
      225,   132,   144,   192,     7,     7,   134,   230,   240,   241,
      144,    82,   144,     5,   144,   144,    82,   190,   230,     5,
-      71,   197,   146,    21,    22,    33,   251,   254,    23,    24,
-     147,   252,    98,    22,   233,    27,   144,   183,   184,   254,
-     189,   144,   198,   253,   254,   179,   254,   191,     7,    46,
-      47,    46,    47,   144,   180,   254,   156,   157,   254,    10,
-      63,   144,   226,   227,   228,   229,   230,   247,   144,   253,
-     226,   134,   238,   239,    61,   241,   242,     7,    53,    76,
-      90,    91,   113,   138,   245,   245,   230,     7,   145,   145,
-     144,   198,   201,   202,   205,   234,   254,   224,   196,   254,
-     231,   232,   144,   254,   254,    23,    58,   145,   230,   243,
-     146,   224,    11,   145,   146,   180,   192,   156,    45,    70,
-     103,   129,   162,   254,   254,   169,   170,   171,   172,   173,
-     174,   254,   144,   165,   145,   146,    82,   158,   229,   198,
-     226,   230,     8,     9,    10,    11,    12,    13,    14,    15,
-      16,    17,    18,    19,    77,   249,    20,   244,   246,   145,
-     122,   230,   239,   242,   230,    62,   145,    71,    71,   145,
-     201,    33,   203,   204,    72,    79,    87,   109,   200,   146,
-     203,    39,   127,   199,    73,   206,    99,   213,   214,   145,
-     243,   145,   146,   184,   230,   254,   144,    85,    85,   144,
-     145,   146,     5,     7,    28,    65,    94,    95,   126,   144,
-     254,   255,   247,   248,   254,   131,   162,   163,   164,   156,
-      10,    45,    55,    92,   103,   108,   129,   159,   160,   161,
-     145,   227,   228,    17,    18,    19,    77,   230,   144,   198,
-     230,    10,    92,   145,   146,   132,   230,   122,    62,   230,
-       7,   145,   204,    97,    97,    97,    84,   201,     7,     7,
-     203,    42,    75,   207,    42,   145,    96,   215,   145,   230,
-     226,   144,   144,   253,   170,   253,   145,   146,    37,    40,
-      41,   116,   175,   146,   137,   166,    92,   144,   244,    85,
-     254,   159,   230,   144,   198,     9,   243,    92,   244,   144,
-     230,   145,    69,   145,   202,   101,   101,   243,   226,    96,
-     208,   243,    42,   106,   112,   216,   217,   145,   253,   253,
-     145,   145,   247,   144,   176,   162,    38,    99,   167,   226,
-     144,     9,   243,   230,   145,   246,     7,    95,    42,    88,
-     209,   220,   221,   230,    19,   145,   145,   169,   144,    42,
-     145,   254,   230,   145,   145,   145,   226,   220,     7,   136,
-     210,   211,   212,   146,    34,    57,   222,     7,    50,   128,
-     218,   108,   145,   169,    74,   106,   168,   145,   254,   212,
-     221,    93,   223,   102,   110,   102,     9,   254,   145,   144,
-      33,    66,    86,     7,    50,   128,   219,   144,   253,   144,
-      68,   110,    68,   253,   145,   213,   145,   100,   145,     7
+      71,   197,   146,    21,    22,    33,   252,   255,    23,    24,
+     147,   253,    98,    22,   233,    27,   144,   183,   184,   255,
+     189,   144,   198,   254,   255,   179,   255,   191,     7,    46,
+      47,    46,    47,   144,   180,   255,   156,   157,   255,    10,
+      63,   144,   226,   227,   228,   229,   230,   248,   144,   247,
+     254,   226,   134,   238,   239,    61,   241,   242,     7,    53,
+      76,    90,    91,   113,   138,   245,   245,   230,     7,   145,
+     145,   144,   198,   201,   202,   205,   234,   255,   224,   196,
+     255,   231,   232,   144,   255,   255,    23,    58,   145,   230,
+     243,   146,   224,    11,   145,   146,   180,   192,   156,    45,
+      70,   103,   129,   162,   255,   255,   169,   170,   171,   172,
+     173,   174,   255,   144,   165,   145,   146,    82,   158,   229,
+     198,   226,   230,     8,     9,    10,    11,    12,    13,    14,
+      15,    16,    17,    18,    19,    77,   250,    20,   244,   246,
+     146,   145,   122,   230,   239,   242,   230,    62,   145,    71,
+      71,   145,   201,    33,   203,   204,    72,    79,    87,   109,
+     200,   146,   203,    39,   127,   199,    73,   206,    99,   213,
+     214,   145,   243,   145,   146,   184,   230,   255,   144,    85,
+      85,   144,   145,   146,     5,     7,    28,    65,    94,    95,
+     126,   144,   255,   256,   248,   249,   255,   131,   162,   163,
+     164,   156,    10,    45,    55,    92,   103,   108,   129,   159,
+     160,   161,   145,   227,   228,    17,    18,    19,    77,   230,
+     144,   198,   230,    10,    92,   145,   146,   144,   132,   230,
+     122,    62,   230,     7,   145,   204,    97,    97,    97,    84,
+     201,     7,     7,   203,    42,    75,   207,    42,   145,    96,
+     215,   145,   230,   226,   144,   144,   254,   170,   254,   145,
+     146,    37,    40,    41,   116,   175,   146,   137,   166,    92,
+     144,   244,    85,   255,   159,   230,   144,   198,     9,   243,
+      92,   244,   246,   144,   230,   145,    69,   145,   202,   101,
+     101,   243,   226,    96,   208,   243,    42,   106,   112,   216,
+     217,   145,   254,   254,   145,   145,   248,   144,   176,   162,
+      38,    99,   167,   226,   144,     9,   243,   230,   145,   145,
+     246,     7,    95,    42,    88,   209,   220,   221,   230,    19,
+     145,   145,   169,   144,    42,   145,   255,   230,   145,   145,
+     145,   226,   220,     7,   136,   210,   211,   212,   146,    34,
+      57,   222,     7,    50,   128,   218,   108,   145,   169,    74,
+     106,   168,   145,   255,   212,   221,    93,   223,   102,   110,
+     102,     9,   255,   145,   144,    33,    66,    86,     7,    50,
+     128,   219,   144,   254,   144,    68,   110,    68,   254,   145,
+     213,   145,   100,   145,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
@@ -1460,9 +1454,10 @@ static const yytype_uint16 yyr1[] =
      237,   237,   238,   238,   239,   240,   240,   241,   242,   242,
      243,   243,   244,   244,   244,   244,   244,   244,   244,   244,
      245,   245,   245,   245,   245,   245,   246,   246,   247,   247,
-     248,   248,   249,   249,   249,   249,   249,   249,   249,   249,
-     249,   249,   250,   251,   251,   252,   252,   252,   253,   253,
-     254,   254,   255,   255,   255,   255,   256,   257,   257
+     248,   248,   249,   249,   250,   250,   250,   250,   250,   250,
+     250,   250,   250,   250,   251,   252,   252,   253,   253,   253,
+     254,   254,   255,   255,   256,   256,   256,   256,   257,   258,
+     258
 };
 
   /* YYR2[YYN] -- Number of symbols on the right hand side of rule YYN.  */
@@ -1477,7 +1472,7 @@ static const yytype_uint8 yyr2[] =
       10,     4,     3,     1,     0,     1,     0,     3,     0,     5,
        0,     8,     1,     1,     1,     3,     1,     1,     1,     1,
        2,     2,     2,     4,     2,     2,     1,     1,     1,     1,
-       0,     3,    10,     7,     4,     5,     5,     4,     4,     5,
+       0,     3,    10,     5,     4,     5,     5,     4,     4,     5,
        2,     2,     2,     0,     4,     5,     4,     3,     1,     3,
        2,     3,     0,     3,     2,     1,     3,     3,     4,     1,
        3,     1,    10,     0,     1,     1,     1,     1,     1,     3,
@@ -1494,10 +1489,11 @@ static const yytype_uint8 yyr2[] =
        1,     3,     1,     3,     4,     4,     5,     6,     6,     8,
        5,     4,     1,     2,     4,     1,     2,     4,     0,     2,
        1,     3,     1,     1,     2,     2,     1,     2,     3,     2,
-       1,     1,     1,     1,     1,     1,     1,     3,     1,     3,
-       1,     3,     1,     1,     1,     1,     1,     1,     1,     2,
-       1,     2,     1,     1,     1,     1,     1,     1,     1,     3,
-       1,     1,     1,     1,     1,     1,     2,     2,     0
+       1,     1,     1,     1,     1,     1,     1,     3,     3,     5,
+       1,     3,     1,     3,     1,     1,     1,     1,     1,     1,
+       1,     2,     1,     2,     1,     1,     1,     1,     1,     1,
+       1,     3,     1,     1,     1,     1,     1,     1,     2,     2,
+       0
 };
 
 
@@ -1994,1071 +1990,1081 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
   switch (yytype)
     {
           case 3: /* TOKEN_COMMAND  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2004 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2000 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 4: /* TOKEN_NAME  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2014 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2010 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 5: /* TOKEN_STRING_SINGLE_QUOTED  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2024 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2020 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 6: /* TOKEN_STRING_DOUBLE_QUOTED  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2034 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2030 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 7: /* TOKEN_UNSIGNED_NUMVAL  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).numeric_literal_value_) != nullptr) {
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 2044 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2040 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 150: /* sql_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2054 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2050 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 151: /* quit_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).quit_statement_) != nullptr) {
     delete ((*yyvaluep).quit_statement_);
   }
 }
-#line 2064 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2060 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 152: /* alter_table_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2074 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2070 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 153: /* create_table_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).create_table_statement_) != nullptr) {
     delete ((*yyvaluep).create_table_statement_);
   }
 }
-#line 2084 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2080 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 154: /* create_index_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2094 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2090 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 155: /* drop_table_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).drop_table_statement_) != nullptr) {
     delete ((*yyvaluep).drop_table_statement_);
   }
 }
-#line 2104 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2100 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 156: /* column_def  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_definition_) != nullptr) {
     delete ((*yyvaluep).attribute_definition_);
   }
 }
-#line 2114 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2110 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 157: /* column_def_commalist  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_definition_list_) != nullptr) {
     delete ((*yyvaluep).attribute_definition_list_);
   }
 }
-#line 2124 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2120 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 158: /* data_type  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).data_type_) != nullptr) {
     delete ((*yyvaluep).data_type_);
   }
 }
-#line 2134 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2130 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 159: /* column_constraint_def  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_) != nullptr) {
     delete ((*yyvaluep).column_constraint_);
   }
 }
-#line 2144 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2140 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 160: /* column_constraint_def_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_list_) != nullptr) {
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 2154 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2150 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 161: /* opt_column_constraint_def_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_list_) != nullptr) {
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 2164 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2160 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 165: /* opt_column_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_list_) != nullptr) {
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 2174 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2170 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 166: /* opt_block_properties  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).block_properties_) != nullptr) {
     delete ((*yyvaluep).block_properties_);
   }
 }
-#line 2184 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2180 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 167: /* opt_partition_clause  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).partition_clause_) != nullptr) {
     delete ((*yyvaluep).partition_clause_);
   }
 }
-#line 2194 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2190 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 168: /* partition_type  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2204 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2200 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 169: /* key_value_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2214 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2210 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 170: /* key_value  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_) != nullptr) {
     delete ((*yyvaluep).key_value_);
   }
 }
-#line 2224 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2220 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 171: /* key_string_value  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_string_value_) != nullptr) {
     delete ((*yyvaluep).key_string_value_);
   }
 }
-#line 2234 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2230 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 172: /* key_string_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_string_list_) != nullptr) {
     delete ((*yyvaluep).key_string_list_);
   }
 }
-#line 2244 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2240 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 173: /* key_integer_value  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_integer_value_) != nullptr) {
     delete ((*yyvaluep).key_integer_value_);
   }
 }
-#line 2254 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2250 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 174: /* key_bool_value  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_bool_value_) != nullptr) {
     delete ((*yyvaluep).key_bool_value_);
   }
 }
-#line 2264 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2260 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 175: /* index_type  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2274 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2270 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 176: /* opt_index_properties  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2284 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2280 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 177: /* insert_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).insert_statement_) != nullptr) {
     delete ((*yyvaluep).insert_statement_);
   }
 }
-#line 2294 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2290 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 178: /* copy_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).copy_statement_) != nullptr) {
     delete ((*yyvaluep).copy_statement_);
   }
 }
-#line 2304 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2300 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 179: /* copy_to_target  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2314 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2310 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 180: /* opt_copy_params  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2324 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2320 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 181: /* update_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).update_statement_) != nullptr) {
     delete ((*yyvaluep).update_statement_);
   }
 }
-#line 2334 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2330 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 182: /* delete_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).delete_statement_) != nullptr) {
     delete ((*yyvaluep).delete_statement_);
   }
 }
-#line 2344 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2340 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 183: /* assignment_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).assignment_list_) != nullptr) {
     delete ((*yyvaluep).assignment_list_);
   }
 }
-#line 2354 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2350 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 184: /* assignment_item  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).assignment_) != nullptr) {
     delete ((*yyvaluep).assignment_);
   }
 }
-#line 2364 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2360 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 185: /* set_operation_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).set_operation_statement_) != nullptr) {
     delete ((*yyvaluep).set_operation_statement_);
   }
 }
-#line 2374 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2370 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 186: /* opt_priority_clause  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).opt_priority_clause_) != nullptr) {
     delete ((*yyvaluep).opt_priority_clause_);
   }
 }
-#line 2384 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2380 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 187: /* with_clause  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).with_list_) != nullptr) {
     delete ((*yyvalu

<TRUNCATED>


[3/3] incubator-quickstep git commit: Support Multiple Tuple Inserts

Posted by ji...@apache.org.
Support Multiple Tuple Inserts

Update Fetch


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/0fe838df
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/0fe838df
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/0fe838df

Branch: refs/heads/master
Commit: 0fe838dfeac901ff03b8334da46b7b9f364447e3
Parents: 79bfcf9
Author: Robert Claus <ro...@gmail.com>
Authored: Tue Oct 24 18:08:57 2017 -0500
Committer: Robert Claus <ro...@gmail.com>
Committed: Wed Oct 25 13:24:02 2017 -0500

----------------------------------------------------------------------
 parser/ParseStatement.hpp                |   22 +-
 parser/SqlParser.ypp                     |   18 +-
 parser/preprocessed/SqlParser_gen.cpp    | 2646 +++++++++++++------------
 parser/preprocessed/SqlParser_gen.hpp    |    3 +-
 query_optimizer/ExecutionGenerator.cpp   |  110 +-
 query_optimizer/logical/InsertTuple.cpp  |    6 +-
 query_optimizer/logical/InsertTuple.hpp  |   10 +-
 query_optimizer/physical/InsertTuple.cpp |    6 +-
 query_optimizer/physical/InsertTuple.hpp |    8 +-
 query_optimizer/resolver/Resolver.cpp    |  122 +-
 10 files changed, 1503 insertions(+), 1448 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/parser/ParseStatement.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseStatement.hpp b/parser/ParseStatement.hpp
index cee7221..456bdc2 100644
--- a/parser/ParseStatement.hpp
+++ b/parser/ParseStatement.hpp
@@ -653,9 +653,9 @@ class ParseStatementInsertTuple : public ParseStatementInsert {
   ParseStatementInsertTuple(const int line_number,
                             const int column_number,
                             const ParseString *relation_name,
-                            PtrList<ParseScalarLiteral> *literal_values)
+                            PtrList<PtrList<ParseScalarLiteral>> *literal_values_list)
       : ParseStatementInsert(line_number, column_number, relation_name),
-        literal_values_(literal_values) {
+        literal_values_(literal_values_list) {
   }
 
   ~ParseStatementInsertTuple() override {
@@ -666,11 +666,11 @@ class ParseStatementInsertTuple : public ParseStatementInsert {
   }
 
   /**
-   * @brief Get the parsed literal attribute values to insert.
+   * @brief Get the list of list of parsed literal attribute values to insert.
    *
-   * @return The list of literal values to insert.
+   * @return The list of lists of literal values to insert.
    **/
-  const PtrList<ParseScalarLiteral>& getLiteralValues() const {
+  const PtrList<PtrList<ParseScalarLiteral>>& getLiteralValues() const {
     return *literal_values_;
   }
 
@@ -685,15 +685,17 @@ class ParseStatementInsertTuple : public ParseStatementInsert {
     inline_field_names->push_back("relation_name");
     inline_field_values->push_back(relation_name()->value());
 
-    container_child_field_names->push_back("tuple");
-    container_child_fields->emplace_back();
-    for (const ParseScalarLiteral& literal_value : *literal_values_) {
-      container_child_fields->back().push_back(&literal_value);
+    for (const PtrList<ParseScalarLiteral>& literal_values_single_tuple : *literal_values_) {
+      container_child_field_names->push_back("tuple");
+      container_child_fields->emplace_back();
+      for (const ParseScalarLiteral& literal_value : literal_values_single_tuple) {
+        container_child_fields->back().push_back(&literal_value);
+      }
     }
   }
 
  private:
-  std::unique_ptr<PtrList<ParseScalarLiteral> > literal_values_;
+  std::unique_ptr<PtrList<PtrList<ParseScalarLiteral>>> literal_values_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseStatementInsertTuple);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 8fbcdd7..ba69b3d 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -128,6 +128,7 @@ typedef void* yyscan_t;
   quickstep::NumericParseLiteralValue *numeric_literal_value_;
   quickstep::ParseLiteralValue *literal_value_;
   quickstep::PtrList<quickstep::ParseScalarLiteral> *literal_value_list_;
+  quickstep::PtrList<quickstep::PtrList<quickstep::ParseScalarLiteral>> *literal_value_list_multiple_;
 
   quickstep::ParseExpression *expression_;
 
@@ -387,6 +388,9 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %type <literal_value_list_>
   literal_value_commalist
 
+%type <literal_value_list_multiple_>
+  literal_value_commalist_multiple
+
 %type <expression_>
   expression_base
   unary_expression
@@ -1101,8 +1105,8 @@ insert_statement:
     NotSupported(&@4, yyscanner, "list of column names in INSERT statement");
     YYERROR;
   }
-  | TOKEN_INSERT TOKEN_INTO any_name TOKEN_VALUES '(' literal_value_commalist ')' {
-    $$ = new quickstep::ParseStatementInsertTuple(@1.first_line, @1.first_column, $3, $6);
+  | TOKEN_INSERT TOKEN_INTO any_name TOKEN_VALUES literal_value_commalist_multiple {
+    $$ = new quickstep::ParseStatementInsertTuple(@1.first_line, @1.first_column, $3, $5);
   }
   | TOKEN_INSERT TOKEN_INTO any_name select_query {
     $$ = new quickstep::ParseStatementInsertSelection(@1.first_line, @2.first_column, $3, $4, nullptr);
@@ -1921,6 +1925,16 @@ literal_value_commalist:
     $$->push_back(new quickstep::ParseScalarLiteral($3));
   };
 
+literal_value_commalist_multiple:
+  '(' literal_value_commalist ')' {
+    $$ = new quickstep::PtrList<quickstep::PtrList<quickstep::ParseScalarLiteral>>();
+    $$->push_back($2);
+  }
+  | literal_value_commalist_multiple ',' '(' literal_value_commalist ')' {
+    $$ = $1;
+    $$->push_back($4);
+  };
+
 attribute_ref:
   any_name {
     $$ = new quickstep::ParseAttribute(@1.first_line, @1.first_column, $1);