diff --git a/src/duckdb/extension/parquet/column_reader.cpp b/src/duckdb/extension/parquet/column_reader.cpp index 2c13b5f46..2b243c958 100644 --- a/src/duckdb/extension/parquet/column_reader.cpp +++ b/src/duckdb/extension/parquet/column_reader.cpp @@ -182,7 +182,7 @@ idx_t ColumnReader::GroupRowsAvailable() { unique_ptr ColumnReader::Stats(idx_t row_group_idx_p, const vector &columns) { if (Type().id() == LogicalTypeId::LIST || Type().id() == LogicalTypeId::STRUCT || - Type().id() == LogicalTypeId::MAP) { + Type().id() == LogicalTypeId::MAP || Type().id() == LogicalTypeId::ARRAY) { return nullptr; } return ParquetStatisticsUtils::TransformColumnStatistics(Schema(), Type(), columns[file_idx]); diff --git a/src/duckdb/extension/parquet/include/parquet_metadata.hpp b/src/duckdb/extension/parquet/include/parquet_metadata.hpp index f3666b5e1..79b4696a9 100644 --- a/src/duckdb/extension/parquet/include/parquet_metadata.hpp +++ b/src/duckdb/extension/parquet/include/parquet_metadata.hpp @@ -23,4 +23,9 @@ class ParquetSchemaFunction : public TableFunction { ParquetSchemaFunction(); }; +class ParquetKeyValueMetadataFunction : public TableFunction { +public: + ParquetKeyValueMetadataFunction(); +}; + } // namespace duckdb diff --git a/src/duckdb/extension/parquet/include/parquet_reader.hpp b/src/duckdb/extension/parquet/include/parquet_reader.hpp index 736e5935a..a7ffd59b8 100644 --- a/src/duckdb/extension/parquet/include/parquet_reader.hpp +++ b/src/duckdb/extension/parquet/include/parquet_reader.hpp @@ -10,16 +10,16 @@ #include "duckdb.hpp" #ifndef DUCKDB_AMALGAMATION -#include "duckdb/planner/table_filter.hpp" -#include "duckdb/planner/filter/constant_filter.hpp" -#include "duckdb/planner/filter/null_filter.hpp" -#include "duckdb/planner/filter/conjunction_filter.hpp" #include "duckdb/common/common.hpp" #include "duckdb/common/exception.hpp" +#include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/common/multi_file_reader_options.hpp" #include "duckdb/common/string_util.hpp" #include "duckdb/common/types/data_chunk.hpp" -#include "duckdb/common/multi_file_reader_options.hpp" -#include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/planner/filter/conjunction_filter.hpp" +#include "duckdb/planner/filter/constant_filter.hpp" +#include "duckdb/planner/filter/null_filter.hpp" +#include "duckdb/planner/table_filter.hpp" #endif #include "column_reader.hpp" #include "parquet_file_metadata_cache.hpp" @@ -64,6 +64,21 @@ struct ParquetReaderScanState { bool current_group_prefetched = false; }; +struct ParquetColumnDefinition { +public: + static ParquetColumnDefinition FromSchemaValue(ClientContext &context, const Value &column_value); + +public: + int32_t field_id; + string name; + LogicalType type; + Value default_value; + +public: + void Serialize(Serializer &serializer) const; + static ParquetColumnDefinition Deserialize(Deserializer &deserializer); +}; + struct ParquetOptions { explicit ParquetOptions() { } @@ -72,6 +87,7 @@ struct ParquetOptions { bool binary_as_string = false; bool file_row_number = false; MultiFileReaderOptions file_options; + vector schema; public: void Serialize(Serializer &serializer) const; @@ -95,6 +111,11 @@ class ParquetReader { MultiFileReaderData reader_data; unique_ptr root_reader; + //! Index of the file_row_number column + idx_t file_row_number_idx = DConstants::INVALID_INDEX; + //! Parquet schema for the generated columns + vector generated_column_schema; + public: void InitializeScan(ParquetReaderScanState &state, vector groups_to_read); void Scan(ParquetReaderScanState &state, DataChunk &output); diff --git a/src/duckdb/extension/parquet/include/parquet_writer.hpp b/src/duckdb/extension/parquet/include/parquet_writer.hpp index 5295a4bca..97c9aa1ea 100644 --- a/src/duckdb/extension/parquet/include/parquet_writer.hpp +++ b/src/duckdb/extension/parquet/include/parquet_writer.hpp @@ -26,6 +26,9 @@ namespace duckdb { class FileSystem; class FileOpener; +class Serializer; +class Deserializer; + struct PreparedRowGroup { duckdb_parquet::format::RowGroup row_group; vector> states; @@ -37,6 +40,9 @@ struct ChildFieldIDs { ChildFieldIDs(); ChildFieldIDs Copy() const; unique_ptr> ids; + + void Serialize(Serializer &serializer) const; + static ChildFieldIDs Deserialize(Deserializer &source); }; struct FieldID { @@ -47,12 +53,16 @@ struct FieldID { bool set; int32_t field_id; ChildFieldIDs child_field_ids; + + void Serialize(Serializer &serializer) const; + static FieldID Deserialize(Deserializer &source); }; class ParquetWriter { public: ParquetWriter(FileSystem &fs, string file_name, vector types, vector names, - duckdb_parquet::format::CompressionCodec::type codec, ChildFieldIDs field_ids); + duckdb_parquet::format::CompressionCodec::type codec, ChildFieldIDs field_ids, + const vector> &kv_metadata); public: void PrepareRowGroup(ColumnDataCollection &buffer, PreparedRowGroup &result); diff --git a/src/duckdb/extension/parquet/parquet_extension.cpp b/src/duckdb/extension/parquet/parquet_extension.cpp index d61ddafa7..9e01c71a5 100644 --- a/src/duckdb/extension/parquet/parquet_extension.cpp +++ b/src/duckdb/extension/parquet/parquet_extension.cpp @@ -2,10 +2,12 @@ #include "parquet_extension.hpp" +#include "cast_column_reader.hpp" #include "duckdb.hpp" #include "parquet_metadata.hpp" #include "parquet_reader.hpp" #include "parquet_writer.hpp" +#include "struct_column_reader.hpp" #include "zstd_file_system.hpp" #include @@ -118,6 +120,7 @@ struct ParquetWriteBindData : public TableFunctionData { vector sql_types; vector column_names; duckdb_parquet::format::CompressionCodec::type codec = duckdb_parquet::format::CompressionCodec::SNAPPY; + vector> kv_metadata; idx_t row_group_size = Storage::ROW_GROUP_SIZE; //! If row_group_size_bytes is not set, we default to row_group_size * BYTES_PER_ROW @@ -157,6 +160,125 @@ BindInfo ParquetGetBatchInfo(const FunctionData *bind_data) { return bind_info; } +static MultiFileReaderBindData BindSchema(ClientContext &context, vector &return_types, + vector &names, ParquetReadBindData &result, ParquetOptions &options) { + D_ASSERT(!options.schema.empty()); + auto &file_options = options.file_options; + if (file_options.union_by_name || file_options.hive_partitioning) { + throw BinderException("Parquet schema cannot be combined with union_by_name=true or hive_partitioning=true"); + } + + vector schema_col_names; + vector schema_col_types; + schema_col_names.reserve(options.schema.size()); + schema_col_types.reserve(options.schema.size()); + for (const auto &column : options.schema) { + schema_col_names.push_back(column.name); + schema_col_types.push_back(column.type); + } + + // perform the binding on the obtained set of names + types + auto bind_data = + MultiFileReader::BindOptions(options.file_options, result.files, schema_col_types, schema_col_names); + + names = schema_col_names; + return_types = schema_col_types; + D_ASSERT(names.size() == return_types.size()); + + if (options.file_row_number) { + if (std::find(names.begin(), names.end(), "file_row_number") != names.end()) { + throw BinderException( + "Using file_row_number option on file with column named file_row_number is not supported"); + } + + bind_data.file_row_number_idx = names.size(); + return_types.emplace_back(LogicalType::BIGINT); + names.emplace_back("file_row_number"); + } + + return bind_data; +} + +static void InitializeParquetReader(ParquetReader &reader, const ParquetReadBindData &bind_data, + const vector &global_column_ids, + optional_ptr table_filters, ClientContext &context) { + auto &parquet_options = bind_data.parquet_options; + auto &reader_data = reader.reader_data; + if (bind_data.parquet_options.schema.empty()) { + MultiFileReader::InitializeReader(reader, parquet_options.file_options, bind_data.reader_bind, bind_data.types, + bind_data.names, global_column_ids, table_filters, bind_data.files[0], + context); + return; + } + + // a fixed schema was supplied, initialize the MultiFileReader settings here so we can read using the schema + + // this deals with hive partitioning and filename=true + MultiFileReader::FinalizeBind(parquet_options.file_options, bind_data.reader_bind, reader.GetFileName(), + reader.GetNames(), bind_data.types, bind_data.names, global_column_ids, reader_data, + context); + + // create a mapping from field id to column index in file + unordered_map field_id_to_column_index; + auto &column_readers = reader.root_reader->Cast().child_readers; + for (idx_t column_index = 0; column_index < column_readers.size(); column_index++) { + auto &column_schema = column_readers[column_index]->Schema(); + if (column_schema.__isset.field_id) { + field_id_to_column_index[column_schema.field_id] = column_index; + } + } + + // loop through the schema definition + for (idx_t i = 0; i < global_column_ids.size(); i++) { + auto global_column_index = global_column_ids[i]; + + // check if this is a constant column + bool constant = false; + for (auto &entry : reader_data.constant_map) { + if (entry.column_id == i) { + constant = true; + break; + } + } + if (constant) { + // this column is constant for this file + continue; + } + + // Handle any generate columns that are not in the schema (currently only file_row_number) + if (global_column_index >= parquet_options.schema.size()) { + if (bind_data.reader_bind.file_row_number_idx == global_column_index) { + reader_data.column_mapping.push_back(i); + reader_data.column_ids.push_back(reader.file_row_number_idx); + } + continue; + } + + const auto &column_definition = parquet_options.schema[global_column_index]; + auto it = field_id_to_column_index.find(column_definition.field_id); + if (it == field_id_to_column_index.end()) { + // field id not present in file, use default value + reader_data.constant_map.emplace_back(i, column_definition.default_value); + continue; + } + + const auto &local_column_index = it->second; + auto &column_reader = column_readers[local_column_index]; + if (column_reader->Type() != column_definition.type) { + // differing types, wrap in a cast column reader + reader_data.cast_map[local_column_index] = column_definition.type; + } + + reader_data.column_mapping.push_back(i); + reader_data.column_ids.push_back(local_column_index); + } + reader_data.empty_columns = reader_data.column_ids.empty(); + + // Finally, initialize the filters + MultiFileReader::CreateFilterMap(bind_data.types, table_filters, reader_data); + reader_data.filters = table_filters; +} + class ParquetScanFunction { public: static TableFunctionSet GetFunctionSet() { @@ -168,6 +290,10 @@ class ParquetScanFunction { table_function.named_parameters["binary_as_string"] = LogicalType::BOOLEAN; table_function.named_parameters["file_row_number"] = LogicalType::BOOLEAN; table_function.named_parameters["compression"] = LogicalType::VARCHAR; + table_function.named_parameters["schema"] = + LogicalType::MAP(LogicalType::INTEGER, LogicalType::STRUCT({{{"name", LogicalType::VARCHAR}, + {"type", LogicalType::VARCHAR}, + {"default_value", LogicalType::VARCHAR}}})); MultiFileReader::AddParameters(table_function); table_function.get_batch_index = ParquetScanGetBatchIndex; table_function.serialize = ParquetScanSerialize; @@ -271,8 +397,14 @@ class ParquetScanFunction { ParquetOptions parquet_options) { auto result = make_uniq(); result->files = std::move(files); - result->reader_bind = - MultiFileReader::BindReader(context, result->types, result->names, *result, parquet_options); + if (parquet_options.schema.empty()) { + result->reader_bind = MultiFileReader::BindReader(context, result->types, result->names, + *result, parquet_options); + } else { + // a schema was supplied + result->reader_bind = BindSchema(context, result->types, result->names, *result, parquet_options); + } + if (return_types.empty()) { // no expected types - just copy the types return_types = result->types; @@ -286,6 +418,7 @@ class ParquetScanFunction { // expected types - overwrite the types we want to read instead result->types = return_types; } + result->parquet_options = parquet_options; return std::move(result); } @@ -302,6 +435,21 @@ class ParquetScanFunction { parquet_options.binary_as_string = BooleanValue::Get(kv.second); } else if (loption == "file_row_number") { parquet_options.file_row_number = BooleanValue::Get(kv.second); + } else if (loption == "schema") { + // Argument is a map that defines the schema + const auto &schema_value = kv.second; + const auto column_values = ListValue::GetChildren(schema_value); + if (column_values.empty()) { + throw BinderException("Parquet schema cannot be empty"); + } + parquet_options.schema.reserve(column_values.size()); + for (idx_t i = 0; i < column_values.size(); i++) { + parquet_options.schema.emplace_back( + ParquetColumnDefinition::FromSchemaValue(context, column_values[i])); + } + + // cannot be combined with hive_partitioning=true, so we disable auto-detection + parquet_options.file_options.auto_detect_hive_partitioning = false; } } parquet_options.file_options.AutoDetectHivePartitioning(files, context); @@ -372,9 +520,7 @@ class ParquetScanFunction { if (!reader) { continue; } - MultiFileReader::InitializeReader(*reader, bind_data.parquet_options.file_options, bind_data.reader_bind, - bind_data.types, bind_data.names, input.column_ids, input.filters, - bind_data.files[0], context); + InitializeParquetReader(*reader, bind_data, input.column_ids, input.filters, context); } result->column_ids = input.column_ids; @@ -569,10 +715,8 @@ class ParquetScanFunction { shared_ptr reader; try { reader = make_shared(context, file, pq_options); - MultiFileReader::InitializeReader(*reader, bind_data.parquet_options.file_options, - bind_data.reader_bind, bind_data.types, bind_data.names, - parallel_state.column_ids, parallel_state.filters, - bind_data.files.front(), context); + InitializeParquetReader(*reader, bind_data, parallel_state.column_ids, parallel_state.filters, + context); } catch (...) { parallel_lock.lock(); parallel_state.error_opening_file = true; @@ -792,6 +936,24 @@ unique_ptr ParquetWriteBind(ClientContext &context, const CopyInfo } GetFieldIDs(option.second[0], bind_data->field_ids, unique_field_ids, name_to_type_map); } + } else if (loption == "kv_metadata") { + auto &kv_struct = option.second[0]; + auto &kv_struct_type = kv_struct.type(); + if (kv_struct_type.id() != LogicalTypeId::STRUCT) { + throw BinderException("Expected kv_metadata argument to be a STRUCT"); + } + auto values = StructValue::GetChildren(kv_struct); + for (idx_t i = 0; i < values.size(); i++) { + auto value = values[i]; + auto key = StructType::GetChildName(kv_struct_type, i); + // If the value is a blob, write the raw blob bytes + // otherwise, cast to string + if (value.type().id() == LogicalTypeId::BLOB) { + bind_data->kv_metadata.emplace_back(key, StringValue::Get(value)); + } else { + bind_data->kv_metadata.emplace_back(key, value.ToString()); + } + } } else { throw NotImplementedException("Unrecognized option for PARQUET: %s", option.first.c_str()); } @@ -811,8 +973,9 @@ unique_ptr ParquetWriteInitializeGlobal(ClientContext &conte auto &parquet_bind = bind_data.Cast(); auto &fs = FileSystem::GetFileSystem(context); - global_state->writer = make_uniq(fs, file_path, parquet_bind.sql_types, parquet_bind.column_names, - parquet_bind.codec, parquet_bind.field_ids.Copy()); + global_state->writer = + make_uniq(fs, file_path, parquet_bind.sql_types, parquet_bind.column_names, parquet_bind.codec, + parquet_bind.field_ids.Copy(), parquet_bind.kv_metadata); return std::move(global_state); } @@ -920,6 +1083,9 @@ static void ParquetCopySerialize(Serializer &serializer, const FunctionData &bin serializer.WriteProperty(101, "column_names", bind_data.column_names); serializer.WriteProperty(102, "codec", bind_data.codec); serializer.WriteProperty(103, "row_group_size", bind_data.row_group_size); + serializer.WriteProperty(104, "row_group_size_bytes", bind_data.row_group_size_bytes); + serializer.WriteProperty(105, "kv_metadata", bind_data.kv_metadata); + serializer.WriteProperty(106, "field_ids", bind_data.field_ids); } static unique_ptr ParquetCopyDeserialize(Deserializer &deserializer, CopyFunction &function) { @@ -928,6 +1094,9 @@ static unique_ptr ParquetCopyDeserialize(Deserializer &deserialize data->column_names = deserializer.ReadProperty>(101, "column_names"); data->codec = deserializer.ReadProperty(102, "codec"); data->row_group_size = deserializer.ReadProperty(103, "row_group_size"); + data->row_group_size_bytes = deserializer.ReadProperty(104, "row_group_size_bytes"); + data->kv_metadata = deserializer.ReadProperty>>(105, "kv_metadata"); + data->field_ids = deserializer.ReadProperty(106, "field_ids"); return std::move(data); } // LCOV_EXCL_STOP @@ -1018,6 +1187,10 @@ void ParquetExtension::Load(DuckDB &db) { ParquetSchemaFunction schema_fun; ExtensionUtil::RegisterFunction(db_instance, MultiFileReader::CreateFunctionSet(schema_fun)); + // parquet_key_value_metadata + ParquetKeyValueMetadataFunction kv_meta_fun; + ExtensionUtil::RegisterFunction(db_instance, MultiFileReader::CreateFunctionSet(kv_meta_fun)); + CopyFunction function("parquet"); function.copy_to_bind = ParquetWriteBind; function.copy_to_initialize_global = ParquetWriteInitializeGlobal; diff --git a/src/duckdb/extension/parquet/parquet_metadata.cpp b/src/duckdb/extension/parquet/parquet_metadata.cpp index f37194291..29e23092a 100644 --- a/src/duckdb/extension/parquet/parquet_metadata.cpp +++ b/src/duckdb/extension/parquet/parquet_metadata.cpp @@ -24,6 +24,8 @@ struct ParquetMetaDataBindData : public TableFunctionData { } }; +enum class ParquetMetadataOperatorType { META_DATA, SCHEMA, KEY_VALUE_META_DATA }; + struct ParquetMetaDataOperatorData : public GlobalTableFunctionState { explicit ParquetMetaDataOperatorData(ClientContext &context, const vector &types) : collection(context, types) { @@ -36,9 +38,11 @@ struct ParquetMetaDataOperatorData : public GlobalTableFunctionState { public: static void BindMetaData(vector &return_types, vector &names); static void BindSchema(vector &return_types, vector &names); + static void BindKeyValueMetaData(vector &return_types, vector &names); void LoadFileMetaData(ClientContext &context, const vector &return_types, const string &file_path); void LoadSchemaData(ClientContext &context, const vector &return_types, const string &file_path); + void LoadKeyValueMetaData(ClientContext &context, const vector &return_types, const string &file_path); }; template @@ -429,13 +433,53 @@ void ParquetMetaDataOperatorData::LoadSchemaData(ClientContext &context, const v collection.InitializeScan(scan_state); } -template +void ParquetMetaDataOperatorData::BindKeyValueMetaData(vector &return_types, vector &names) { + names.emplace_back("key"); + return_types.emplace_back(LogicalType::BLOB); + + names.emplace_back("value"); + return_types.emplace_back(LogicalType::BLOB); +} + +void ParquetMetaDataOperatorData::LoadKeyValueMetaData(ClientContext &context, const vector &return_types, + const string &file_path) { + collection.Reset(); + ParquetOptions parquet_options(context); + auto reader = make_uniq(context, file_path, parquet_options); + idx_t count = 0; + DataChunk current_chunk; + current_chunk.Initialize(context, return_types); + auto meta_data = reader->GetFileMetadata(); + + for (idx_t col_idx = 0; col_idx < meta_data->key_value_metadata.size(); col_idx++) { + auto &entry = meta_data->key_value_metadata[col_idx]; + + current_chunk.SetValue(0, count, Value::BLOB_RAW(entry.key)); + current_chunk.SetValue(1, count, Value::BLOB_RAW(entry.value)); + + count++; + if (count >= STANDARD_VECTOR_SIZE) { + current_chunk.SetCardinality(count); + collection.Append(current_chunk); + + count = 0; + current_chunk.Reset(); + } + } + current_chunk.SetCardinality(count); + collection.Append(current_chunk); + collection.InitializeScan(scan_state); +} + +template unique_ptr ParquetMetaDataBind(ClientContext &context, TableFunctionBindInput &input, vector &return_types, vector &names) { - if (SCHEMA) { + if (TYPE == ParquetMetadataOperatorType::SCHEMA) { ParquetMetaDataOperatorData::BindSchema(return_types, names); - } else { + } else if (TYPE == ParquetMetadataOperatorType::META_DATA) { ParquetMetaDataOperatorData::BindMetaData(return_types, names); + } else if (TYPE == ParquetMetadataOperatorType::KEY_VALUE_META_DATA) { + ParquetMetaDataOperatorData::BindKeyValueMetaData(return_types, names); } auto result = make_uniq(); @@ -444,22 +488,24 @@ unique_ptr ParquetMetaDataBind(ClientContext &context, TableFuncti return std::move(result); } -template +template unique_ptr ParquetMetaDataInit(ClientContext &context, TableFunctionInitInput &input) { auto &bind_data = input.bind_data->Cast(); D_ASSERT(!bind_data.files.empty()); auto result = make_uniq(context, bind_data.return_types); - if (SCHEMA) { + if (TYPE == ParquetMetadataOperatorType::SCHEMA) { result->LoadSchemaData(context, bind_data.return_types, bind_data.files[0]); - } else { + } else if (TYPE == ParquetMetadataOperatorType::META_DATA) { result->LoadFileMetaData(context, bind_data.return_types, bind_data.files[0]); + } else if (TYPE == ParquetMetadataOperatorType::KEY_VALUE_META_DATA) { + result->LoadKeyValueMetaData(context, bind_data.return_types, bind_data.files[0]); } result->file_index = 0; return std::move(result); } -template +template void ParquetMetaDataImplementation(ClientContext &context, TableFunctionInput &data_p, DataChunk &output) { auto &data = data_p.global_state->Cast(); auto &bind_data = data_p.bind_data->Cast(); @@ -469,10 +515,12 @@ void ParquetMetaDataImplementation(ClientContext &context, TableFunctionInput &d if (data.file_index + 1 < bind_data.files.size()) { // load the metadata for the next file data.file_index++; - if (SCHEMA) { + if (TYPE == ParquetMetadataOperatorType::SCHEMA) { data.LoadSchemaData(context, bind_data.return_types, bind_data.files[data.file_index]); - } else { + } else if (TYPE == ParquetMetadataOperatorType::META_DATA) { data.LoadFileMetaData(context, bind_data.return_types, bind_data.files[data.file_index]); + } else if (TYPE == ParquetMetadataOperatorType::KEY_VALUE_META_DATA) { + data.LoadKeyValueMetaData(context, bind_data.return_types, bind_data.files[data.file_index]); } continue; } else { @@ -487,13 +535,24 @@ void ParquetMetaDataImplementation(ClientContext &context, TableFunctionInput &d } ParquetMetaDataFunction::ParquetMetaDataFunction() - : TableFunction("parquet_metadata", {LogicalType::VARCHAR}, ParquetMetaDataImplementation, - ParquetMetaDataBind, ParquetMetaDataInit) { + : TableFunction("parquet_metadata", {LogicalType::VARCHAR}, + ParquetMetaDataImplementation, + ParquetMetaDataBind, + ParquetMetaDataInit) { } ParquetSchemaFunction::ParquetSchemaFunction() - : TableFunction("parquet_schema", {LogicalType::VARCHAR}, ParquetMetaDataImplementation, - ParquetMetaDataBind, ParquetMetaDataInit) { + : TableFunction("parquet_schema", {LogicalType::VARCHAR}, + ParquetMetaDataImplementation, + ParquetMetaDataBind, + ParquetMetaDataInit) { +} + +ParquetKeyValueMetadataFunction::ParquetKeyValueMetadataFunction() + : TableFunction("parquet_kv_metadata", {LogicalType::VARCHAR}, + ParquetMetaDataImplementation, + ParquetMetaDataBind, + ParquetMetaDataInit) { } } // namespace duckdb diff --git a/src/duckdb/extension/parquet/parquet_reader.cpp b/src/duckdb/extension/parquet/parquet_reader.cpp index f6f5723ae..75e35b3d7 100644 --- a/src/duckdb/extension/parquet/parquet_reader.cpp +++ b/src/duckdb/extension/parquet/parquet_reader.cpp @@ -1,41 +1,37 @@ #include "parquet_reader.hpp" -#include "parquet_timestamp.hpp" -#include "parquet_statistics.hpp" -#include "column_reader.hpp" #include "boolean_column_reader.hpp" -#include "row_number_column_reader.hpp" -#include "cast_column_reader.hpp" #include "callback_column_reader.hpp" +#include "cast_column_reader.hpp" +#include "column_reader.hpp" +#include "duckdb.hpp" #include "list_column_reader.hpp" +#include "parquet_file_metadata_cache.hpp" +#include "parquet_statistics.hpp" +#include "parquet_timestamp.hpp" +#include "row_number_column_reader.hpp" #include "string_column_reader.hpp" #include "struct_column_reader.hpp" #include "templated_column_reader.hpp" - #include "thrift_tools.hpp" - -#include "parquet_file_metadata_cache.hpp" - -#include "duckdb.hpp" #ifndef DUCKDB_AMALGAMATION -#include "duckdb/planner/table_filter.hpp" -#include "duckdb/planner/filter/constant_filter.hpp" -#include "duckdb/planner/filter/null_filter.hpp" -#include "duckdb/planner/filter/conjunction_filter.hpp" #include "duckdb/common/file_system.hpp" +#include "duckdb/common/hive_partitioning.hpp" +#include "duckdb/common/pair.hpp" #include "duckdb/common/string_util.hpp" #include "duckdb/common/types/date.hpp" -#include "duckdb/common/pair.hpp" -#include "duckdb/common/hive_partitioning.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" - +#include "duckdb/planner/filter/conjunction_filter.hpp" +#include "duckdb/planner/filter/constant_filter.hpp" +#include "duckdb/planner/filter/null_filter.hpp" +#include "duckdb/planner/table_filter.hpp" #include "duckdb/storage/object_cache.hpp" #endif -#include #include #include #include +#include namespace duckdb { @@ -385,8 +381,11 @@ unique_ptr ParquetReader::CreateReader() { root_struct_reader.child_readers[column_idx] = std::move(cast_reader); } if (parquet_options.file_row_number) { - root_struct_reader.child_readers.push_back( - make_uniq(*this, LogicalType::BIGINT, SchemaElement(), next_file_idx, 0, 0)); + file_row_number_idx = root_struct_reader.child_readers.size(); + + generated_column_schema.push_back(SchemaElement()); + root_struct_reader.child_readers.push_back(make_uniq( + *this, LogicalType::BIGINT, generated_column_schema.back(), next_file_idx, 0, 0)); } return ret; @@ -429,6 +428,25 @@ ParquetOptions::ParquetOptions(ClientContext &context) { } } +ParquetColumnDefinition ParquetColumnDefinition::FromSchemaValue(ClientContext &context, const Value &column_value) { + ParquetColumnDefinition result; + result.field_id = IntegerValue::Get(StructValue::GetChildren(column_value)[0]); + + const auto &column_def = StructValue::GetChildren(column_value)[1]; + D_ASSERT(column_def.type().id() == LogicalTypeId::STRUCT); + + const auto children = StructValue::GetChildren(column_def); + result.name = StringValue::Get(children[0]); + result.type = TransformStringToLogicalType(StringValue::Get(children[1])); + string error_message; + if (!children[2].TryCastAs(context, result.type, result.default_value, &error_message)) { + throw BinderException("Unable to cast Parquet schema default_value \"%s\" to %s", children[2].ToString(), + result.type.ToString()); + } + + return result; +} + ParquetReader::ParquetReader(ClientContext &context_p, string file_name_p, ParquetOptions parquet_options_p) : fs(FileSystem::GetFileSystem(context_p)), allocator(BufferAllocator::Get(context_p)), parquet_options(std::move(parquet_options_p)) { diff --git a/src/duckdb/extension/parquet/parquet_writer.cpp b/src/duckdb/extension/parquet/parquet_writer.cpp index a80dc0dbb..149a6b071 100644 --- a/src/duckdb/extension/parquet/parquet_writer.cpp +++ b/src/duckdb/extension/parquet/parquet_writer.cpp @@ -30,8 +30,7 @@ using duckdb_parquet::format::PageType; using ParquetRowGroup = duckdb_parquet::format::RowGroup; using duckdb_parquet::format::Type; -ChildFieldIDs::ChildFieldIDs() { - ids = make_uniq>(); +ChildFieldIDs::ChildFieldIDs() : ids(make_uniq>()) { } ChildFieldIDs ChildFieldIDs::Copy() const { @@ -323,7 +322,8 @@ void VerifyUniqueNames(const vector &names) { } ParquetWriter::ParquetWriter(FileSystem &fs, string file_name_p, vector types_p, vector names_p, - CompressionCodec::type codec, ChildFieldIDs field_ids_p) + CompressionCodec::type codec, ChildFieldIDs field_ids_p, + const vector> &kv_metadata) : file_name(std::move(file_name_p)), sql_types(std::move(types_p)), column_names(std::move(names_p)), codec(codec), field_ids(std::move(field_ids_p)) { // initialize the file writer @@ -342,6 +342,14 @@ ParquetWriter::ParquetWriter(FileSystem &fs, string file_name_p, vector>>(100, "ids", ids); +} + +ChildFieldIDs ChildFieldIDs::Deserialize(Deserializer &deserializer) { + ChildFieldIDs result; + deserializer.ReadPropertyWithDefault>>(100, "ids", result.ids); + return result; +} + +void FieldID::Serialize(Serializer &serializer) const { + serializer.WritePropertyWithDefault(100, "set", set); + serializer.WritePropertyWithDefault(101, "field_id", field_id); + serializer.WriteProperty(102, "child_field_ids", child_field_ids); +} + +FieldID FieldID::Deserialize(Deserializer &deserializer) { + FieldID result; + deserializer.ReadPropertyWithDefault(100, "set", result.set); + deserializer.ReadPropertyWithDefault(101, "field_id", result.field_id); + deserializer.ReadProperty(102, "child_field_ids", result.child_field_ids); + return result; +} + +void ParquetColumnDefinition::Serialize(Serializer &serializer) const { + serializer.WritePropertyWithDefault(100, "field_id", field_id); + serializer.WritePropertyWithDefault(101, "name", name); + serializer.WriteProperty(103, "type", type); + serializer.WriteProperty(104, "default_value", default_value); +} + +ParquetColumnDefinition ParquetColumnDefinition::Deserialize(Deserializer &deserializer) { + ParquetColumnDefinition result; + deserializer.ReadPropertyWithDefault(100, "field_id", result.field_id); + deserializer.ReadPropertyWithDefault(101, "name", result.name); + deserializer.ReadProperty(103, "type", result.type); + deserializer.ReadProperty(104, "default_value", result.default_value); + return result; +} + void ParquetOptions::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault(100, "binary_as_string", binary_as_string); serializer.WritePropertyWithDefault(101, "file_row_number", file_row_number); serializer.WriteProperty(102, "file_options", file_options); + serializer.WritePropertyWithDefault>(103, "schema", schema); } ParquetOptions ParquetOptions::Deserialize(Deserializer &deserializer) { @@ -20,6 +64,7 @@ ParquetOptions ParquetOptions::Deserialize(Deserializer &deserializer) { deserializer.ReadPropertyWithDefault(100, "binary_as_string", result.binary_as_string); deserializer.ReadPropertyWithDefault(101, "file_row_number", result.file_row_number); deserializer.ReadProperty(102, "file_options", result.file_options); + deserializer.ReadPropertyWithDefault>(103, "schema", result.schema); return result; } diff --git a/src/duckdb/src/catalog/catalog_entry/schema_catalog_entry.cpp b/src/duckdb/src/catalog/catalog_entry/schema_catalog_entry.cpp index 7b1062528..c2e3a7279 100644 --- a/src/duckdb/src/catalog/catalog_entry/schema_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/schema_catalog_entry.cpp @@ -39,9 +39,8 @@ unique_ptr SchemaCatalogEntry::GetInfo() const { } string SchemaCatalogEntry::ToSQL() const { - std::stringstream ss; - ss << "CREATE SCHEMA " << name << ";"; - return ss.str(); + auto create_schema_info = GetInfo(); + return create_schema_info->ToString(); } } // namespace duckdb diff --git a/src/duckdb/src/catalog/catalog_entry/table_catalog_entry.cpp b/src/duckdb/src/catalog/catalog_entry/table_catalog_entry.cpp index 9fd270ac4..a652a6d73 100644 --- a/src/duckdb/src/catalog/catalog_entry/table_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/table_catalog_entry.cpp @@ -151,19 +151,8 @@ string TableCatalogEntry::ColumnsToSQL(const ColumnList &columns, const vectorToString(); } const ColumnList &TableCatalogEntry::GetColumns() const { diff --git a/src/duckdb/src/catalog/catalog_entry/type_catalog_entry.cpp b/src/duckdb/src/catalog/catalog_entry/type_catalog_entry.cpp index 055bdfc7c..f16d85ced 100644 --- a/src/duckdb/src/catalog/catalog_entry/type_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/type_catalog_entry.cpp @@ -25,29 +25,14 @@ unique_ptr TypeCatalogEntry::GetInfo() const { } string TypeCatalogEntry::ToSQL() const { - std::stringstream ss; switch (user_type.id()) { case (LogicalTypeId::ENUM): { - auto &values_insert_order = EnumType::GetValuesInsertOrder(user_type); - idx_t size = EnumType::GetSize(user_type); - ss << "CREATE TYPE "; - ss << KeywordHelper::WriteOptionallyQuoted(name); - ss << " AS ENUM ( "; - - for (idx_t i = 0; i < size; i++) { - ss << "'" << values_insert_order.GetValue(i).ToString() << "'"; - if (i != size - 1) { - ss << ", "; - } - } - ss << ");"; - break; + auto create_type_info = GetInfo(); + return create_type_info->ToString(); } default: throw InternalException("Logical Type can't be used as a User Defined Type"); } - - return ss.str(); } } // namespace duckdb diff --git a/src/duckdb/src/common/adbc/adbc.cpp b/src/duckdb/src/common/adbc/adbc.cpp index 14016adce..4f8467828 100644 --- a/src/duckdb/src/common/adbc/adbc.cpp +++ b/src/duckdb/src/common/adbc/adbc.cpp @@ -8,12 +8,15 @@ #include "duckdb/common/arrow/arrow_wrapper.hpp" #include "duckdb/common/arrow/nanoarrow/nanoarrow.hpp" +#include "duckdb/main/capi/capi_internal.hpp" + #ifndef DUCKDB_AMALGAMATION #include "duckdb/main/connection.hpp" #endif #include "duckdb/common/adbc/single_batch_array_stream.hpp" +#include "duckdb/common/adbc/options.h" #include #include @@ -50,9 +53,6 @@ duckdb_adbc::AdbcStatusCode duckdb_adbc_init(size_t count, struct duckdb_adbc::A driver->StatementGetParameterSchema = duckdb_adbc::StatementGetParameterSchema; driver->ConnectionGetTableSchema = duckdb_adbc::ConnectionGetTableSchema; driver->StatementSetSubstraitPlan = duckdb_adbc::StatementSetSubstraitPlan; - - driver->ConnectionGetInfo = duckdb_adbc::ConnectionGetInfo; - driver->StatementGetParameterSchema = duckdb_adbc::StatementGetParameterSchema; return ADBC_STATUS_OK; } @@ -395,8 +395,8 @@ static AdbcInfoCode ConvertToInfoCode(uint32_t info_code) { } } -AdbcStatusCode ConnectionGetInfo(struct AdbcConnection *connection, uint32_t *info_codes, size_t info_codes_length, - struct ArrowArrayStream *out, struct AdbcError *error) { +AdbcStatusCode ConnectionGetInfo(struct AdbcConnection *connection, const uint32_t *info_codes, + size_t info_codes_length, struct ArrowArrayStream *out, struct AdbcError *error) { if (!connection) { SetError(error, "Missing connection object"); return ADBC_STATUS_INVALID_ARGUMENT; @@ -765,6 +765,9 @@ AdbcStatusCode StatementExecuteQuery(struct AdbcStatement *statement, struct Arr return ADBC_STATUS_INVALID_ARGUMENT; } duckdb::unique_ptr chunk; + auto prepared_statement_params = + reinterpret_cast(wrapper->statement)->statement->n_param; + while ((chunk = result->Fetch()) != nullptr) { if (chunk->size() == 0) { SetError(error, "Please provide a non-empty chunk to be bound"); @@ -775,6 +778,10 @@ AdbcStatusCode StatementExecuteQuery(struct AdbcStatement *statement, struct Arr SetError(error, "Binding multiple rows at once is not supported yet"); return ADBC_STATUS_NOT_IMPLEMENTED; } + if (chunk->ColumnCount() > prepared_statement_params) { + SetError(error, "Input data has more column than prepared statement has parameters"); + return ADBC_STATUS_INVALID_ARGUMENT; + } duckdb_clear_bindings(wrapper->statement); for (idx_t col_idx = 0; col_idx < chunk->ColumnCount(); col_idx++) { auto val = chunk->GetValue(col_idx, 0); @@ -922,6 +929,12 @@ AdbcStatusCode StatementSetOption(struct AdbcStatement *statement, const char *k wrapper->ingestion_table_name = strdup(value); return ADBC_STATUS_OK; } + if (strcmp(key, ADBC_INGEST_OPTION_TEMPORARY) == 0) { + if (strcmp(value, "false") == 0) { + return ADBC_STATUS_NOT_IMPLEMENTED; + } + return ADBC_STATUS_OK; + } if (strcmp(key, ADBC_INGEST_OPTION_MODE) == 0) { if (strcmp(value, ADBC_INGEST_OPTION_MODE_CREATE) == 0) { wrapper->ingestion_mode = IngestionMode::CREATE; diff --git a/src/duckdb/src/common/adbc/driver_manager.cpp b/src/duckdb/src/common/adbc/driver_manager.cpp index 23ae98269..aaabc40e5 100644 --- a/src/duckdb/src/common/adbc/driver_manager.cpp +++ b/src/duckdb/src/common/adbc/driver_manager.cpp @@ -1,3 +1,9 @@ +//////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////// +// THIS FILE IS GENERATED BY apache/arrow, DO NOT EDIT MANUALLY // +//////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////// + // Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information @@ -20,6 +26,8 @@ #include "duckdb/common/adbc/adbc.hpp" #include +#include +#include #include #include #include @@ -57,31 +65,20 @@ void GetWinError(std::string *buffer) { #endif // defined(_WIN32) -// Temporary state while the database is being configured. -struct TempDatabase { - std::unordered_map options; - std::string driver; - // Default name (see adbc.h) - std::string entrypoint = "AdbcDriverInit"; - AdbcDriverInitFunc init_func = nullptr; -}; - // Error handling void ReleaseError(struct AdbcError *error) { if (error) { - if (error->message) { + if (error->message) delete[] error->message; - } error->message = nullptr; error->release = nullptr; } } void SetError(struct AdbcError *error, const std::string &message) { - if (!error) { + if (!error) return; - } if (error->message) { // Append std::string buffer = error->message; @@ -101,61 +98,482 @@ void SetError(struct AdbcError *error, const std::string &message) { error->release = ReleaseError; } -void SetError(struct AdbcError *error, const char *message_p) { - if (!message_p) { - message_p = ""; +// Driver state + +/// A driver DLL. +struct ManagedLibrary { + ManagedLibrary() : handle(nullptr) { + } + ManagedLibrary(ManagedLibrary &&other) : handle(other.handle) { + other.handle = nullptr; + } + ManagedLibrary(const ManagedLibrary &) = delete; + ManagedLibrary &operator=(const ManagedLibrary &) = delete; + ManagedLibrary &operator=(ManagedLibrary &&other) noexcept { + this->handle = other.handle; + other.handle = nullptr; + return *this; } - std::string message(message_p); - SetError(error, message); -} -// Driver state + ~ManagedLibrary() { + Release(); + } -/// Hold the driver DLL and the driver release callback in the driver struct. -struct ManagerDriverState { - // The original release callback - AdbcStatusCode (*driver_release)(struct AdbcDriver *driver, struct AdbcError *error); + void Release() { + // TODO(apache/arrow-adbc#204): causes tests to segfault + // Need to refcount the driver DLL; also, errors may retain a reference to + // release() from the DLL - how to handle this? + } + + AdbcStatusCode Load(const char *library, struct AdbcError *error) { + std::string error_message; +#if defined(_WIN32) + HMODULE handle = LoadLibraryExA(library, NULL, 0); + if (!handle) { + error_message += library; + error_message += ": LoadLibraryExA() failed: "; + GetWinError(&error_message); + + std::string full_driver_name = library; + full_driver_name += ".dll"; + handle = LoadLibraryExA(full_driver_name.c_str(), NULL, 0); + if (!handle) { + error_message += '\n'; + error_message += full_driver_name; + error_message += ": LoadLibraryExA() failed: "; + GetWinError(&error_message); + } + } + if (!handle) { + SetError(error, error_message); + return ADBC_STATUS_INTERNAL; + } else { + this->handle = handle; + } +#else + const std::string kPlatformLibraryPrefix = "lib"; +#if defined(__APPLE__) + const std::string kPlatformLibrarySuffix = ".dylib"; +#else + static const std::string kPlatformLibrarySuffix = ".so"; +#endif // defined(__APPLE__) + + void *handle = dlopen(library, RTLD_NOW | RTLD_LOCAL); + if (!handle) { + error_message = "dlopen() failed: "; + error_message += dlerror(); + + // If applicable, append the shared library prefix/extension and + // try again (this way you don't have to hardcode driver names by + // platform in the application) + const std::string driver_str = library; + + std::string full_driver_name; + if (driver_str.size() < kPlatformLibraryPrefix.size() || + driver_str.compare(0, kPlatformLibraryPrefix.size(), kPlatformLibraryPrefix) != 0) { + full_driver_name += kPlatformLibraryPrefix; + } + full_driver_name += library; + if (driver_str.size() < kPlatformLibrarySuffix.size() || + driver_str.compare(full_driver_name.size() - kPlatformLibrarySuffix.size(), + kPlatformLibrarySuffix.size(), kPlatformLibrarySuffix) != 0) { + full_driver_name += kPlatformLibrarySuffix; + } + handle = dlopen(full_driver_name.c_str(), RTLD_NOW | RTLD_LOCAL); + if (!handle) { + error_message += "\ndlopen() failed: "; + error_message += dlerror(); + } + } + if (handle) { + this->handle = handle; + } else { + return ADBC_STATUS_INTERNAL; + } +#endif // defined(_WIN32) + return ADBC_STATUS_OK; + } + + AdbcStatusCode Lookup(const char *name, void **func, struct AdbcError *error) { +#if defined(_WIN32) + void *load_handle = reinterpret_cast(GetProcAddress(handle, name)); + if (!load_handle) { + std::string message = "GetProcAddress("; + message += name; + message += ") failed: "; + GetWinError(&message); + SetError(error, message); + return ADBC_STATUS_INTERNAL; + } +#else + void *load_handle = dlsym(handle, name); + if (!load_handle) { + std::string message = "dlsym("; + message += name; + message += ") failed: "; + message += dlerror(); + SetError(error, message); + return ADBC_STATUS_INTERNAL; + } +#endif // defined(_WIN32) + *func = load_handle; + return ADBC_STATUS_OK; + } #if defined(_WIN32) // The loaded DLL HMODULE handle; +#else + void *handle; #endif // defined(_WIN32) }; +/// Hold the driver DLL and the driver release callback in the driver struct. +struct ManagerDriverState { + // The original release callback + AdbcStatusCode (*driver_release)(struct AdbcDriver *driver, struct AdbcError *error); + + ManagedLibrary handle; +}; + /// Unload the driver DLL. static AdbcStatusCode ReleaseDriver(struct AdbcDriver *driver, struct AdbcError *error) { AdbcStatusCode status = ADBC_STATUS_OK; - if (!driver->private_manager) { + if (!driver->private_manager) return status; - } ManagerDriverState *state = reinterpret_cast(driver->private_manager); if (state->driver_release) { status = state->driver_release(driver, error); } - -#if defined(_WIN32) - // TODO(apache/arrow-adbc#204): causes tests to segfault - // if (!FreeLibrary(state->handle)) { - // std::string message = "FreeLibrary() failed: "; - // GetWinError(&message); - // SetError(error, message); - // } -#endif // defined(_WIN32) + state->handle.Release(); driver->private_manager = nullptr; delete state; return status; } +// ArrowArrayStream wrapper to support AdbcErrorFromArrayStream + +struct ErrorArrayStream { + struct ArrowArrayStream stream; + struct AdbcDriver *private_driver; +}; + +void ErrorArrayStreamRelease(struct ArrowArrayStream *stream) { + if (stream->release != ErrorArrayStreamRelease || !stream->private_data) + return; + + auto *private_data = reinterpret_cast(stream->private_data); + private_data->stream.release(&private_data->stream); + delete private_data; + std::memset(stream, 0, sizeof(*stream)); +} + +const char *ErrorArrayStreamGetLastError(struct ArrowArrayStream *stream) { + if (stream->release != ErrorArrayStreamRelease || !stream->private_data) + return nullptr; + auto *private_data = reinterpret_cast(stream->private_data); + return private_data->stream.get_last_error(&private_data->stream); +} + +int ErrorArrayStreamGetNext(struct ArrowArrayStream *stream, struct ArrowArray *array) { + if (stream->release != ErrorArrayStreamRelease || !stream->private_data) + return EINVAL; + auto *private_data = reinterpret_cast(stream->private_data); + return private_data->stream.get_next(&private_data->stream, array); +} + +int ErrorArrayStreamGetSchema(struct ArrowArrayStream *stream, struct ArrowSchema *schema) { + if (stream->release != ErrorArrayStreamRelease || !stream->private_data) + return EINVAL; + auto *private_data = reinterpret_cast(stream->private_data); + return private_data->stream.get_schema(&private_data->stream, schema); +} + +// Default stubs + +int ErrorGetDetailCount(const struct AdbcError *error) { + return 0; +} + +struct AdbcErrorDetail ErrorGetDetail(const struct AdbcError *error, int index) { + return {nullptr, nullptr, 0}; +} + +const struct AdbcError *ErrorFromArrayStream(struct ArrowArrayStream *stream, AdbcStatusCode *status) { + return nullptr; +} + +void ErrorArrayStreamInit(struct ArrowArrayStream *out, struct AdbcDriver *private_driver) { + if (!out || !out->release || + // Don't bother wrapping if driver didn't claim support + private_driver->ErrorFromArrayStream == ErrorFromArrayStream) { + return; + } + struct ErrorArrayStream *private_data = new ErrorArrayStream; + private_data->stream = *out; + private_data->private_driver = private_driver; + out->get_last_error = ErrorArrayStreamGetLastError; + out->get_next = ErrorArrayStreamGetNext; + out->get_schema = ErrorArrayStreamGetSchema; + out->release = ErrorArrayStreamRelease; + out->private_data = private_data; +} + +AdbcStatusCode DatabaseGetOption(struct AdbcDatabase *database, const char *key, char *value, size_t *length, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode DatabaseGetOptionBytes(struct AdbcDatabase *database, const char *key, uint8_t *value, size_t *length, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode DatabaseGetOptionInt(struct AdbcDatabase *database, const char *key, int64_t *value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode DatabaseGetOptionDouble(struct AdbcDatabase *database, const char *key, double *value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode DatabaseSetOptionBytes(struct AdbcDatabase *database, const char *key, const uint8_t *value, + size_t length, struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode DatabaseSetOptionInt(struct AdbcDatabase *database, const char *key, int64_t value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode DatabaseSetOptionDouble(struct AdbcDatabase *database, const char *key, double value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode ConnectionCancel(struct AdbcConnection *connection, struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode ConnectionGetOption(struct AdbcConnection *connection, const char *key, char *value, size_t *length, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode ConnectionGetOptionBytes(struct AdbcConnection *connection, const char *key, uint8_t *value, + size_t *length, struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode ConnectionGetOptionInt(struct AdbcConnection *connection, const char *key, int64_t *value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode ConnectionGetOptionDouble(struct AdbcConnection *connection, const char *key, double *value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode ConnectionGetStatistics(struct AdbcConnection *, const char *, const char *, const char *, char, + struct ArrowArrayStream *, struct AdbcError *) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode ConnectionGetStatisticNames(struct AdbcConnection *, struct ArrowArrayStream *, struct AdbcError *) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode ConnectionSetOptionBytes(struct AdbcConnection *, const char *, const uint8_t *, size_t, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode ConnectionSetOptionInt(struct AdbcConnection *connection, const char *key, int64_t value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode ConnectionSetOptionDouble(struct AdbcConnection *connection, const char *key, double value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode StatementCancel(struct AdbcStatement *statement, struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode StatementExecuteSchema(struct AdbcStatement *statement, struct ArrowSchema *schema, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode StatementGetOption(struct AdbcStatement *statement, const char *key, char *value, size_t *length, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode StatementGetOptionBytes(struct AdbcStatement *statement, const char *key, uint8_t *value, size_t *length, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode StatementGetOptionInt(struct AdbcStatement *statement, const char *key, int64_t *value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode StatementGetOptionDouble(struct AdbcStatement *statement, const char *key, double *value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_FOUND; +} + +AdbcStatusCode StatementSetOptionBytes(struct AdbcStatement *, const char *, const uint8_t *, size_t, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode StatementSetOptionInt(struct AdbcStatement *statement, const char *key, int64_t value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +AdbcStatusCode StatementSetOptionDouble(struct AdbcStatement *statement, const char *key, double value, + struct AdbcError *error) { + return ADBC_STATUS_NOT_IMPLEMENTED; +} + +/// Temporary state while the database is being configured. +struct TempDatabase { + std::unordered_map options; + std::unordered_map bytes_options; + std::unordered_map int_options; + std::unordered_map double_options; + std::string driver; + std::string entrypoint; + AdbcDriverInitFunc init_func = nullptr; +}; + /// Temporary state while the database is being configured. struct TempConnection { std::unordered_map options; + std::unordered_map bytes_options; + std::unordered_map int_options; + std::unordered_map double_options; }; +static const char kDefaultEntrypoint[] = "AdbcDriverInit"; + +// Other helpers (intentionally not in an anonymous namespace so they can be tested) + +ADBC_EXPORT +std::string AdbcDriverManagerDefaultEntrypoint(const std::string &driver) { + /// - libadbc_driver_sqlite.so.2.0.0 -> AdbcDriverSqliteInit + /// - adbc_driver_sqlite.dll -> AdbcDriverSqliteInit + /// - proprietary_driver.dll -> AdbcProprietaryDriverInit + + // Potential path -> filename + // Treat both \ and / as directory separators on all platforms for simplicity + std::string filename; + { + size_t pos = driver.find_last_of("/\\"); + if (pos != std::string::npos) { + filename = driver.substr(pos + 1); + } else { + filename = driver; + } + } + + // Remove all extensions + { + size_t pos = filename.find('.'); + if (pos != std::string::npos) { + filename = filename.substr(0, pos); + } + } + + // Remove lib prefix + // https://stackoverflow.com/q/1878001/262727 + if (filename.rfind("lib", 0) == 0) { + filename = filename.substr(3); + } + + // Split on underscores, hyphens + // Capitalize and join + std::string entrypoint; + entrypoint.reserve(filename.size()); + size_t pos = 0; + while (pos < filename.size()) { + size_t prev = pos; + pos = filename.find_first_of("-_", pos); + // if pos == npos this is the entire filename + std::string token = filename.substr(prev, pos - prev); + // capitalize first letter + token[0] = std::toupper(static_cast(token[0])); + + entrypoint += token; + + if (pos != std::string::npos) { + pos++; + } + } + + if (entrypoint.rfind("Adbc", 0) != 0) { + entrypoint = "Adbc" + entrypoint; + } + entrypoint += "Init"; + + return entrypoint; +} + // Direct implementations of API methods +int AdbcErrorGetDetailCount(const struct AdbcError *error) { + if (error->vendor_code == ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA && error->private_data && error->private_driver) { + return error->private_driver->ErrorGetDetailCount(error); + } + return 0; +} + +struct AdbcErrorDetail AdbcErrorGetDetail(const struct AdbcError *error, int index) { + if (error->vendor_code == ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA && error->private_data && error->private_driver) { + return error->private_driver->ErrorGetDetail(error, index); + } + return {nullptr, nullptr, 0}; +} + +const struct AdbcError *AdbcErrorFromArrayStream(struct ArrowArrayStream *stream, AdbcStatusCode *status) { + if (!stream->private_data || stream->release != ErrorArrayStreamRelease) { + return nullptr; + } + auto *private_data = reinterpret_cast(stream->private_data); + auto *error = private_data->private_driver->ErrorFromArrayStream(&private_data->stream, status); + if (error) { + const_cast(error)->private_driver = private_data->private_driver; + } + return error; +} + +#define INIT_ERROR(ERROR, SOURCE) \ + if ((ERROR) != nullptr && (ERROR)->vendor_code == ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA) { \ + (ERROR)->private_driver = (SOURCE)->private_driver; \ + } + +#define WRAP_STREAM(EXPR, OUT, SOURCE) \ + if (!(OUT)) { \ + /* Happens for ExecuteQuery where out is optional */ \ + return EXPR; \ + } \ + AdbcStatusCode status_code = EXPR; \ + ErrorArrayStreamInit(OUT, (SOURCE)->private_driver); \ + return status_code; + AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase *database, struct AdbcError *error) { // Allocate a temporary structure to store options pre-Init database->private_data = new TempDatabase(); @@ -163,12 +581,89 @@ AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase *database, struct AdbcError * return ADBC_STATUS_OK; } -AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase *database, const char *key, const char *value, +AdbcStatusCode AdbcDatabaseGetOption(struct AdbcDatabase *database, const char *key, char *value, size_t *length, struct AdbcError *error) { - if (!database) { - return ADBC_STATUS_INVALID_ARGUMENT; + if (database->private_driver) { + INIT_ERROR(error, database); + return database->private_driver->DatabaseGetOption(database, key, value, length, error); } + const auto *args = reinterpret_cast(database->private_data); + const std::string *result = nullptr; + if (std::strcmp(key, "driver") == 0) { + result = &args->driver; + } else if (std::strcmp(key, "entrypoint") == 0) { + result = &args->entrypoint; + } else { + const auto it = args->options.find(key); + if (it == args->options.end()) { + return ADBC_STATUS_NOT_FOUND; + } + result = &it->second; + } + + if (*length <= result->size() + 1) { + // Enough space + std::memcpy(value, result->c_str(), result->size() + 1); + } + *length = result->size() + 1; + return ADBC_STATUS_OK; +} + +AdbcStatusCode AdbcDatabaseGetOptionBytes(struct AdbcDatabase *database, const char *key, uint8_t *value, + size_t *length, struct AdbcError *error) { + if (database->private_driver) { + INIT_ERROR(error, database); + return database->private_driver->DatabaseGetOptionBytes(database, key, value, length, error); + } + const auto *args = reinterpret_cast(database->private_data); + const auto it = args->bytes_options.find(key); + if (it == args->options.end()) { + return ADBC_STATUS_NOT_FOUND; + } + const std::string &result = it->second; + + if (*length <= result.size()) { + // Enough space + std::memcpy(value, result.c_str(), result.size()); + } + *length = result.size(); + return ADBC_STATUS_OK; +} + +AdbcStatusCode AdbcDatabaseGetOptionInt(struct AdbcDatabase *database, const char *key, int64_t *value, + struct AdbcError *error) { + if (database->private_driver) { + INIT_ERROR(error, database); + return database->private_driver->DatabaseGetOptionInt(database, key, value, error); + } + const auto *args = reinterpret_cast(database->private_data); + const auto it = args->int_options.find(key); + if (it == args->int_options.end()) { + return ADBC_STATUS_NOT_FOUND; + } + *value = it->second; + return ADBC_STATUS_OK; +} + +AdbcStatusCode AdbcDatabaseGetOptionDouble(struct AdbcDatabase *database, const char *key, double *value, + struct AdbcError *error) { if (database->private_driver) { + INIT_ERROR(error, database); + return database->private_driver->DatabaseGetOptionDouble(database, key, value, error); + } + const auto *args = reinterpret_cast(database->private_data); + const auto it = args->double_options.find(key); + if (it == args->double_options.end()) { + return ADBC_STATUS_NOT_FOUND; + } + *value = it->second; + return ADBC_STATUS_OK; +} + +AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase *database, const char *key, const char *value, + struct AdbcError *error) { + if (database->private_driver) { + INIT_ERROR(error, database); return database->private_driver->DatabaseSetOption(database, key, value, error); } @@ -183,11 +678,44 @@ AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase *database, const char * return ADBC_STATUS_OK; } +AdbcStatusCode AdbcDatabaseSetOptionBytes(struct AdbcDatabase *database, const char *key, const uint8_t *value, + size_t length, struct AdbcError *error) { + if (database->private_driver) { + INIT_ERROR(error, database); + return database->private_driver->DatabaseSetOptionBytes(database, key, value, length, error); + } + + TempDatabase *args = reinterpret_cast(database->private_data); + args->bytes_options[key] = std::string(reinterpret_cast(value), length); + return ADBC_STATUS_OK; +} + +AdbcStatusCode AdbcDatabaseSetOptionInt(struct AdbcDatabase *database, const char *key, int64_t value, + struct AdbcError *error) { + if (database->private_driver) { + INIT_ERROR(error, database); + return database->private_driver->DatabaseSetOptionInt(database, key, value, error); + } + + TempDatabase *args = reinterpret_cast(database->private_data); + args->int_options[key] = value; + return ADBC_STATUS_OK; +} + +AdbcStatusCode AdbcDatabaseSetOptionDouble(struct AdbcDatabase *database, const char *key, double value, + struct AdbcError *error) { + if (database->private_driver) { + INIT_ERROR(error, database); + return database->private_driver->DatabaseSetOptionDouble(database, key, value, error); + } + + TempDatabase *args = reinterpret_cast(database->private_data); + args->double_options[key] = value; + return ADBC_STATUS_OK; +} + AdbcStatusCode AdbcDriverManagerDatabaseSetInitFunc(struct AdbcDatabase *database, AdbcDriverInitFunc init_func, struct AdbcError *error) { - if (!database) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (database->private_driver) { return ADBC_STATUS_INVALID_STATE; } @@ -216,10 +744,12 @@ AdbcStatusCode AdbcDatabaseInit(struct AdbcDatabase *database, struct AdbcError // So we don't confuse a driver into thinking it's initialized already database->private_data = nullptr; if (args->init_func) { - status = AdbcLoadDriverFromInitFunc(args->init_func, ADBC_VERSION_1_0_0, database->private_driver, error); - } else { - status = AdbcLoadDriver(args->driver.c_str(), args->entrypoint.c_str(), ADBC_VERSION_1_0_0, + status = AdbcLoadDriverFromInitFunc(args->init_func, ADBC_VERSION_1_1_0, database->private_driver, error); + } else if (!args->entrypoint.empty()) { + status = AdbcLoadDriver(args->driver.c_str(), args->entrypoint.c_str(), ADBC_VERSION_1_1_0, database->private_driver, error); + } else { + status = AdbcLoadDriver(args->driver.c_str(), nullptr, ADBC_VERSION_1_1_0, database->private_driver, error); } if (status != ADBC_STATUS_OK) { // Restore private_data so it will be released by AdbcDatabaseRelease @@ -240,25 +770,51 @@ AdbcStatusCode AdbcDatabaseInit(struct AdbcDatabase *database, struct AdbcError database->private_driver = nullptr; return status; } - for (const auto &option : args->options) { + auto options = std::move(args->options); + auto bytes_options = std::move(args->bytes_options); + auto int_options = std::move(args->int_options); + auto double_options = std::move(args->double_options); + delete args; + + INIT_ERROR(error, database); + for (const auto &option : options) { status = database->private_driver->DatabaseSetOption(database, option.first.c_str(), option.second.c_str(), error); - if (status != ADBC_STATUS_OK) { - delete args; - // Release the database - std::ignore = database->private_driver->DatabaseRelease(database, error); - if (database->private_driver->release) { - database->private_driver->release(database->private_driver, error); - } - delete database->private_driver; - database->private_driver = nullptr; - // Should be redundant, but ensure that AdbcDatabaseRelease - // below doesn't think that it contains a TempDatabase - database->private_data = nullptr; - return status; + if (status != ADBC_STATUS_OK) + break; + } + for (const auto &option : bytes_options) { + status = database->private_driver->DatabaseSetOptionBytes( + database, option.first.c_str(), reinterpret_cast(option.second.data()), + option.second.size(), error); + if (status != ADBC_STATUS_OK) + break; + } + for (const auto &option : int_options) { + status = database->private_driver->DatabaseSetOptionInt(database, option.first.c_str(), option.second, error); + if (status != ADBC_STATUS_OK) + break; + } + for (const auto &option : double_options) { + status = + database->private_driver->DatabaseSetOptionDouble(database, option.first.c_str(), option.second, error); + if (status != ADBC_STATUS_OK) + break; + } + + if (status != ADBC_STATUS_OK) { + // Release the database + std::ignore = database->private_driver->DatabaseRelease(database, error); + if (database->private_driver->release) { + database->private_driver->release(database->private_driver, error); } + delete database->private_driver; + database->private_driver = nullptr; + // Should be redundant, but ensure that AdbcDatabaseRelease + // below doesn't think that it contains a TempDatabase + database->private_data = nullptr; + return status; } - delete args; return database->private_driver->DatabaseInit(database, error); } @@ -272,6 +828,7 @@ AdbcStatusCode AdbcDatabaseRelease(struct AdbcDatabase *database, struct AdbcErr } return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, database); auto status = database->private_driver->DatabaseRelease(database, error); if (database->private_driver->release) { database->private_driver->release(database->private_driver, error); @@ -282,72 +839,174 @@ AdbcStatusCode AdbcDatabaseRelease(struct AdbcDatabase *database, struct AdbcErr return status; } -AdbcStatusCode AdbcConnectionCommit(struct AdbcConnection *connection, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; +AdbcStatusCode AdbcConnectionCancel(struct AdbcConnection *connection, struct AdbcError *error) { + if (!connection->private_driver) { + return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, connection); + return connection->private_driver->ConnectionCancel(connection, error); +} + +AdbcStatusCode AdbcConnectionCommit(struct AdbcConnection *connection, struct AdbcError *error) { if (!connection->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, connection); return connection->private_driver->ConnectionCommit(connection, error); } -AdbcStatusCode AdbcConnectionGetInfo(struct AdbcConnection *connection, uint32_t *info_codes, size_t info_codes_length, - struct ArrowArrayStream *out, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } +AdbcStatusCode AdbcConnectionGetInfo(struct AdbcConnection *connection, const uint32_t *info_codes, + size_t info_codes_length, struct ArrowArrayStream *out, struct AdbcError *error) { if (!connection->private_driver) { return ADBC_STATUS_INVALID_STATE; } - return connection->private_driver->ConnectionGetInfo(connection, info_codes, info_codes_length, out, error); + INIT_ERROR(error, connection); + WRAP_STREAM(connection->private_driver->ConnectionGetInfo(connection, info_codes, info_codes_length, out, error), + out, connection); } AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection *connection, int depth, const char *catalog, const char *db_schema, const char *table_name, const char **table_types, const char *column_name, struct ArrowArrayStream *stream, struct AdbcError *error) { - if (!connection) { - SetError(error, "connection can't be null"); + if (!connection->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, connection); + WRAP_STREAM(connection->private_driver->ConnectionGetObjects(connection, depth, catalog, db_schema, table_name, + table_types, column_name, stream, error), + stream, connection); +} + +AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection *connection, const char *key, char *value, size_t *length, + struct AdbcError *error) { + if (!connection->private_data) { + SetError(error, "AdbcConnectionGetOption: must AdbcConnectionNew first"); + return ADBC_STATUS_INVALID_STATE; + } + if (!connection->private_driver) { + // Init not yet called, get the saved option + const auto *args = reinterpret_cast(connection->private_data); + const auto it = args->options.find(key); + if (it == args->options.end()) { + return ADBC_STATUS_NOT_FOUND; + } + if (*length >= it->second.size() + 1) { + std::memcpy(value, it->second.c_str(), it->second.size() + 1); + } + *length = it->second.size() + 1; + return ADBC_STATUS_OK; + } + INIT_ERROR(error, connection); + return connection->private_driver->ConnectionGetOption(connection, key, value, length, error); +} + +AdbcStatusCode AdbcConnectionGetOptionBytes(struct AdbcConnection *connection, const char *key, uint8_t *value, + size_t *length, struct AdbcError *error) { + if (!connection->private_data) { + SetError(error, "AdbcConnectionGetOption: must AdbcConnectionNew first"); + return ADBC_STATUS_INVALID_STATE; + } + if (!connection->private_driver) { + // Init not yet called, get the saved option + const auto *args = reinterpret_cast(connection->private_data); + const auto it = args->bytes_options.find(key); + if (it == args->options.end()) { + return ADBC_STATUS_NOT_FOUND; + } + if (*length >= it->second.size() + 1) { + std::memcpy(value, it->second.data(), it->second.size() + 1); + } + *length = it->second.size() + 1; + return ADBC_STATUS_OK; + } + INIT_ERROR(error, connection); + return connection->private_driver->ConnectionGetOptionBytes(connection, key, value, length, error); +} + +AdbcStatusCode AdbcConnectionGetOptionInt(struct AdbcConnection *connection, const char *key, int64_t *value, + struct AdbcError *error) { + if (!connection->private_data) { + SetError(error, "AdbcConnectionGetOption: must AdbcConnectionNew first"); return ADBC_STATUS_INVALID_STATE; } + if (!connection->private_driver) { + // Init not yet called, get the saved option + const auto *args = reinterpret_cast(connection->private_data); + const auto it = args->int_options.find(key); + if (it == args->int_options.end()) { + return ADBC_STATUS_NOT_FOUND; + } + *value = it->second; + return ADBC_STATUS_OK; + } + INIT_ERROR(error, connection); + return connection->private_driver->ConnectionGetOptionInt(connection, key, value, error); +} + +AdbcStatusCode AdbcConnectionGetOptionDouble(struct AdbcConnection *connection, const char *key, double *value, + struct AdbcError *error) { if (!connection->private_data) { - SetError(error, "connection must be initialized"); + SetError(error, "AdbcConnectionGetOption: must AdbcConnectionNew first"); + return ADBC_STATUS_INVALID_STATE; + } + if (!connection->private_driver) { + // Init not yet called, get the saved option + const auto *args = reinterpret_cast(connection->private_data); + const auto it = args->double_options.find(key); + if (it == args->double_options.end()) { + return ADBC_STATUS_NOT_FOUND; + } + *value = it->second; + return ADBC_STATUS_OK; + } + INIT_ERROR(error, connection); + return connection->private_driver->ConnectionGetOptionDouble(connection, key, value, error); +} + +AdbcStatusCode AdbcConnectionGetStatistics(struct AdbcConnection *connection, const char *catalog, + const char *db_schema, const char *table_name, char approximate, + struct ArrowArrayStream *out, struct AdbcError *error) { + if (!connection->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, connection); + WRAP_STREAM(connection->private_driver->ConnectionGetStatistics(connection, catalog, db_schema, table_name, + approximate == 1, out, error), + out, connection); +} + +AdbcStatusCode AdbcConnectionGetStatisticNames(struct AdbcConnection *connection, struct ArrowArrayStream *out, + struct AdbcError *error) { + if (!connection->private_driver) { return ADBC_STATUS_INVALID_STATE; } - return connection->private_driver->ConnectionGetObjects(connection, depth, catalog, db_schema, table_name, - table_types, column_name, stream, error); + INIT_ERROR(error, connection); + WRAP_STREAM(connection->private_driver->ConnectionGetStatisticNames(connection, out, error), out, connection); } AdbcStatusCode AdbcConnectionGetTableSchema(struct AdbcConnection *connection, const char *catalog, const char *db_schema, const char *table_name, struct ArrowSchema *schema, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!connection->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, connection); return connection->private_driver->ConnectionGetTableSchema(connection, catalog, db_schema, table_name, schema, error); } AdbcStatusCode AdbcConnectionGetTableTypes(struct AdbcConnection *connection, struct ArrowArrayStream *stream, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!connection->private_driver) { return ADBC_STATUS_INVALID_STATE; } - return connection->private_driver->ConnectionGetTableTypes(connection, stream, error); + INIT_ERROR(error, connection); + WRAP_STREAM(connection->private_driver->ConnectionGetTableTypes(connection, stream, error), stream, connection); } AdbcStatusCode AdbcConnectionInit(struct AdbcConnection *connection, struct AdbcDatabase *database, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!connection->private_data) { SetError(error, "Must call AdbcConnectionNew first"); return ADBC_STATUS_INVALID_STATE; @@ -358,21 +1017,42 @@ AdbcStatusCode AdbcConnectionInit(struct AdbcConnection *connection, struct Adbc TempConnection *args = reinterpret_cast(connection->private_data); connection->private_data = nullptr; std::unordered_map options = std::move(args->options); + std::unordered_map bytes_options = std::move(args->bytes_options); + std::unordered_map int_options = std::move(args->int_options); + std::unordered_map double_options = std::move(args->double_options); delete args; auto status = database->private_driver->ConnectionNew(connection, error); - if (status != ADBC_STATUS_OK) { + if (status != ADBC_STATUS_OK) return status; - } connection->private_driver = database->private_driver; for (const auto &option : options) { status = database->private_driver->ConnectionSetOption(connection, option.first.c_str(), option.second.c_str(), error); - if (status != ADBC_STATUS_OK) { + if (status != ADBC_STATUS_OK) + return status; + } + for (const auto &option : bytes_options) { + status = database->private_driver->ConnectionSetOptionBytes( + connection, option.first.c_str(), reinterpret_cast(option.second.data()), + option.second.size(), error); + if (status != ADBC_STATUS_OK) + return status; + } + for (const auto &option : int_options) { + status = + database->private_driver->ConnectionSetOptionInt(connection, option.first.c_str(), option.second, error); + if (status != ADBC_STATUS_OK) + return status; + } + for (const auto &option : double_options) { + status = + database->private_driver->ConnectionSetOptionDouble(connection, option.first.c_str(), option.second, error); + if (status != ADBC_STATUS_OK) return status; - } } + INIT_ERROR(error, connection); return connection->private_driver->ConnectionInit(connection, database, error); } @@ -388,20 +1068,16 @@ AdbcStatusCode AdbcConnectionNew(struct AdbcConnection *connection, struct AdbcE AdbcStatusCode AdbcConnectionReadPartition(struct AdbcConnection *connection, const uint8_t *serialized_partition, size_t serialized_length, struct ArrowArrayStream *out, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!connection->private_driver) { return ADBC_STATUS_INVALID_STATE; } - return connection->private_driver->ConnectionReadPartition(connection, serialized_partition, serialized_length, out, - error); + INIT_ERROR(error, connection); + WRAP_STREAM(connection->private_driver->ConnectionReadPartition(connection, serialized_partition, serialized_length, + out, error), + out, connection); } AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection *connection, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!connection->private_driver) { if (connection->private_data) { TempConnection *args = reinterpret_cast(connection->private_data); @@ -411,26 +1087,22 @@ AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection *connection, struct A } return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, connection); auto status = connection->private_driver->ConnectionRelease(connection, error); connection->private_driver = nullptr; return status; } AdbcStatusCode AdbcConnectionRollback(struct AdbcConnection *connection, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!connection->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, connection); return connection->private_driver->ConnectionRollback(connection, error); } AdbcStatusCode AdbcConnectionSetOption(struct AdbcConnection *connection, const char *key, const char *value, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!connection->private_data) { SetError(error, "AdbcConnectionSetOption: must AdbcConnectionNew first"); return ADBC_STATUS_INVALID_STATE; @@ -441,98 +1113,182 @@ AdbcStatusCode AdbcConnectionSetOption(struct AdbcConnection *connection, const args->options[key] = value; return ADBC_STATUS_OK; } + INIT_ERROR(error, connection); return connection->private_driver->ConnectionSetOption(connection, key, value, error); } +AdbcStatusCode AdbcConnectionSetOptionBytes(struct AdbcConnection *connection, const char *key, const uint8_t *value, + size_t length, struct AdbcError *error) { + if (!connection->private_data) { + SetError(error, "AdbcConnectionSetOptionInt: must AdbcConnectionNew first"); + return ADBC_STATUS_INVALID_STATE; + } + if (!connection->private_driver) { + // Init not yet called, save the option + TempConnection *args = reinterpret_cast(connection->private_data); + args->bytes_options[key] = std::string(reinterpret_cast(value), length); + return ADBC_STATUS_OK; + } + INIT_ERROR(error, connection); + return connection->private_driver->ConnectionSetOptionBytes(connection, key, value, length, error); +} + +AdbcStatusCode AdbcConnectionSetOptionInt(struct AdbcConnection *connection, const char *key, int64_t value, + struct AdbcError *error) { + if (!connection->private_data) { + SetError(error, "AdbcConnectionSetOptionInt: must AdbcConnectionNew first"); + return ADBC_STATUS_INVALID_STATE; + } + if (!connection->private_driver) { + // Init not yet called, save the option + TempConnection *args = reinterpret_cast(connection->private_data); + args->int_options[key] = value; + return ADBC_STATUS_OK; + } + INIT_ERROR(error, connection); + return connection->private_driver->ConnectionSetOptionInt(connection, key, value, error); +} + +AdbcStatusCode AdbcConnectionSetOptionDouble(struct AdbcConnection *connection, const char *key, double value, + struct AdbcError *error) { + if (!connection->private_data) { + SetError(error, "AdbcConnectionSetOptionDouble: must AdbcConnectionNew first"); + return ADBC_STATUS_INVALID_STATE; + } + if (!connection->private_driver) { + // Init not yet called, save the option + TempConnection *args = reinterpret_cast(connection->private_data); + args->double_options[key] = value; + return ADBC_STATUS_OK; + } + INIT_ERROR(error, connection); + return connection->private_driver->ConnectionSetOptionDouble(connection, key, value, error); +} + AdbcStatusCode AdbcStatementBind(struct AdbcStatement *statement, struct ArrowArray *values, struct ArrowSchema *schema, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); return statement->private_driver->StatementBind(statement, values, schema, error); } AdbcStatusCode AdbcStatementBindStream(struct AdbcStatement *statement, struct ArrowArrayStream *stream, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); return statement->private_driver->StatementBindStream(statement, stream, error); } +AdbcStatusCode AdbcStatementCancel(struct AdbcStatement *statement, struct AdbcError *error) { + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, statement); + return statement->private_driver->StatementCancel(statement, error); +} + // XXX: cpplint gets confused here if declared as 'struct ArrowSchema* schema' AdbcStatusCode AdbcStatementExecutePartitions(struct AdbcStatement *statement, ArrowSchema *schema, struct AdbcPartitions *partitions, int64_t *rows_affected, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); return statement->private_driver->StatementExecutePartitions(statement, schema, partitions, rows_affected, error); } AdbcStatusCode AdbcStatementExecuteQuery(struct AdbcStatement *statement, struct ArrowArrayStream *out, int64_t *rows_affected, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, statement); + WRAP_STREAM(statement->private_driver->StatementExecuteQuery(statement, out, rows_affected, error), out, statement); +} + +AdbcStatusCode AdbcStatementExecuteSchema(struct AdbcStatement *statement, struct ArrowSchema *schema, + struct AdbcError *error) { + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, statement); + return statement->private_driver->StatementExecuteSchema(statement, schema, error); +} + +AdbcStatusCode AdbcStatementGetOption(struct AdbcStatement *statement, const char *key, char *value, size_t *length, + struct AdbcError *error) { + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, statement); + return statement->private_driver->StatementGetOption(statement, key, value, length, error); +} + +AdbcStatusCode AdbcStatementGetOptionBytes(struct AdbcStatement *statement, const char *key, uint8_t *value, + size_t *length, struct AdbcError *error) { + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, statement); + return statement->private_driver->StatementGetOptionBytes(statement, key, value, length, error); +} + +AdbcStatusCode AdbcStatementGetOptionInt(struct AdbcStatement *statement, const char *key, int64_t *value, + struct AdbcError *error) { + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); + return statement->private_driver->StatementGetOptionInt(statement, key, value, error); +} + +AdbcStatusCode AdbcStatementGetOptionDouble(struct AdbcStatement *statement, const char *key, double *value, + struct AdbcError *error) { if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } - return statement->private_driver->StatementExecuteQuery(statement, out, rows_affected, error); + INIT_ERROR(error, statement); + return statement->private_driver->StatementGetOptionDouble(statement, key, value, error); } AdbcStatusCode AdbcStatementGetParameterSchema(struct AdbcStatement *statement, struct ArrowSchema *schema, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); return statement->private_driver->StatementGetParameterSchema(statement, schema, error); } AdbcStatusCode AdbcStatementNew(struct AdbcConnection *connection, struct AdbcStatement *statement, struct AdbcError *error) { - if (!connection) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!connection->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, connection); auto status = connection->private_driver->StatementNew(connection, statement, error); statement->private_driver = connection->private_driver; return status; } AdbcStatusCode AdbcStatementPrepare(struct AdbcStatement *statement, struct AdbcError *error) { - if (!statement) { - SetError(error, "Missing statement object"); - return ADBC_STATUS_INVALID_ARGUMENT; - } - if (!statement->private_data) { - SetError(error, "Invalid statement object"); + if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); return statement->private_driver->StatementPrepare(statement, error); } AdbcStatusCode AdbcStatementRelease(struct AdbcStatement *statement, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); auto status = statement->private_driver->StatementRelease(statement, error); statement->private_driver = nullptr; return status; @@ -540,65 +1296,82 @@ AdbcStatusCode AdbcStatementRelease(struct AdbcStatement *statement, struct Adbc AdbcStatusCode AdbcStatementSetOption(struct AdbcStatement *statement, const char *key, const char *value, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); return statement->private_driver->StatementSetOption(statement, key, value, error); } -AdbcStatusCode AdbcStatementSetSqlQuery(struct AdbcStatement *statement, const char *query, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; +AdbcStatusCode AdbcStatementSetOptionBytes(struct AdbcStatement *statement, const char *key, const uint8_t *value, + size_t length, struct AdbcError *error) { + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, statement); + return statement->private_driver->StatementSetOptionBytes(statement, key, value, length, error); +} + +AdbcStatusCode AdbcStatementSetOptionInt(struct AdbcStatement *statement, const char *key, int64_t value, + struct AdbcError *error) { + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); + return statement->private_driver->StatementSetOptionInt(statement, key, value, error); +} + +AdbcStatusCode AdbcStatementSetOptionDouble(struct AdbcStatement *statement, const char *key, double value, + struct AdbcError *error) { + if (!statement->private_driver) { + return ADBC_STATUS_INVALID_STATE; + } + INIT_ERROR(error, statement); + return statement->private_driver->StatementSetOptionDouble(statement, key, value, error); +} + +AdbcStatusCode AdbcStatementSetSqlQuery(struct AdbcStatement *statement, const char *query, struct AdbcError *error) { if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); return statement->private_driver->StatementSetSqlQuery(statement, query, error); } AdbcStatusCode AdbcStatementSetSubstraitPlan(struct AdbcStatement *statement, const uint8_t *plan, size_t length, struct AdbcError *error) { - if (!statement) { - return ADBC_STATUS_INVALID_ARGUMENT; - } if (!statement->private_driver) { return ADBC_STATUS_INVALID_STATE; } + INIT_ERROR(error, statement); return statement->private_driver->StatementSetSubstraitPlan(statement, plan, length, error); } const char *AdbcStatusCodeMessage(AdbcStatusCode code) { -#define STRINGIFY(s) #s -#define STRINGIFY_VALUE(s) STRINGIFY(s) #define CASE(CONSTANT) \ - case CONSTANT: \ - return #CONSTANT " (" STRINGIFY_VALUE(CONSTANT) ")"; + case ADBC_STATUS_##CONSTANT: \ + return #CONSTANT; switch (code) { - CASE(ADBC_STATUS_OK); - CASE(ADBC_STATUS_UNKNOWN); - CASE(ADBC_STATUS_NOT_IMPLEMENTED); - CASE(ADBC_STATUS_NOT_FOUND); - CASE(ADBC_STATUS_ALREADY_EXISTS); - CASE(ADBC_STATUS_INVALID_ARGUMENT); - CASE(ADBC_STATUS_INVALID_STATE); - CASE(ADBC_STATUS_INVALID_DATA); - CASE(ADBC_STATUS_INTEGRITY); - CASE(ADBC_STATUS_INTERNAL); - CASE(ADBC_STATUS_IO); - CASE(ADBC_STATUS_CANCELLED); - CASE(ADBC_STATUS_TIMEOUT); - CASE(ADBC_STATUS_UNAUTHENTICATED); - CASE(ADBC_STATUS_UNAUTHORIZED); + CASE(OK); + CASE(UNKNOWN); + CASE(NOT_IMPLEMENTED); + CASE(NOT_FOUND); + CASE(ALREADY_EXISTS); + CASE(INVALID_ARGUMENT); + CASE(INVALID_STATE); + CASE(INVALID_DATA); + CASE(INTEGRITY); + CASE(INTERNAL); + CASE(IO); + CASE(CANCELLED); + CASE(TIMEOUT); + CASE(UNAUTHENTICATED); + CASE(UNAUTHORIZED); default: return "(invalid code)"; } #undef CASE -#undef STRINGIFY_VALUE -#undef STRINGIFY } AdbcStatusCode AdbcLoadDriver(const char *driver_name, const char *entrypoint, int version, void *raw_driver, @@ -606,136 +1379,80 @@ AdbcStatusCode AdbcLoadDriver(const char *driver_name, const char *entrypoint, i AdbcDriverInitFunc init_func; std::string error_message; - if (version != ADBC_VERSION_1_0_0) { - SetError(error, "Only ADBC 1.0.0 is supported"); + switch (version) { + case ADBC_VERSION_1_0_0: + case ADBC_VERSION_1_1_0: + break; + default: + SetError(error, "Only ADBC 1.0.0 and 1.1.0 are supported"); return ADBC_STATUS_NOT_IMPLEMENTED; } - auto *driver = reinterpret_cast(raw_driver); - - if (!entrypoint) { - // Default entrypoint (see adbc.h) - entrypoint = "AdbcDriverInit"; - } - -#if defined(_WIN32) - - HMODULE handle = LoadLibraryExA(driver_name, NULL, 0); - if (!handle) { - error_message += driver_name; - error_message += ": LoadLibraryExA() failed: "; - GetWinError(&error_message); - - std::string full_driver_name = driver_name; - full_driver_name += ".lib"; - handle = LoadLibraryExA(full_driver_name.c_str(), NULL, 0); - if (!handle) { - error_message += '\n'; - error_message += full_driver_name; - error_message += ": LoadLibraryExA() failed: "; - GetWinError(&error_message); - } - } - if (!handle) { - SetError(error, error_message); - return ADBC_STATUS_INTERNAL; + if (!raw_driver) { + SetError(error, "Must provide non-NULL raw_driver"); + return ADBC_STATUS_INVALID_ARGUMENT; } + auto *driver = reinterpret_cast(raw_driver); - void *load_handle = reinterpret_cast(GetProcAddress(handle, entrypoint)); - init_func = reinterpret_cast(load_handle); - if (!init_func) { - std::string message = "GetProcAddress("; - message += entrypoint; - message += ") failed: "; - GetWinError(&message); - if (!FreeLibrary(handle)) { - message += "\nFreeLibrary() failed: "; - GetWinError(&message); - } - SetError(error, message); - return ADBC_STATUS_INTERNAL; + ManagedLibrary library; + AdbcStatusCode status = library.Load(driver_name, error); + if (status != ADBC_STATUS_OK) { + // AdbcDatabaseInit tries to call this if set + driver->release = nullptr; + return status; } -#else - -#if defined(__APPLE__) - const std::string kPlatformLibraryPrefix = "lib"; - const std::string kPlatformLibrarySuffix = ".dylib"; -#else - const std::string kPlatformLibraryPrefix = "lib"; - const std::string kPlatformLibrarySuffix = ".so"; -#endif // defined(__APPLE__) - - void *handle = dlopen(driver_name, RTLD_NOW | RTLD_LOCAL); - if (!handle) { - error_message = "dlopen() failed: "; - error_message += dlerror(); - - // If applicable, append the shared library prefix/extension and - // try again (this way you don't have to hardcode driver names by - // platform in the application) - const std::string driver_str = driver_name; - - std::string full_driver_name; - if (driver_str.size() < kPlatformLibraryPrefix.size() || - driver_str.compare(0, kPlatformLibraryPrefix.size(), kPlatformLibraryPrefix) != 0) { - full_driver_name += kPlatformLibraryPrefix; - } - full_driver_name += driver_name; - if (driver_str.size() < kPlatformLibrarySuffix.size() || - driver_str.compare(full_driver_name.size() - kPlatformLibrarySuffix.size(), kPlatformLibrarySuffix.size(), - kPlatformLibrarySuffix) != 0) { - full_driver_name += kPlatformLibrarySuffix; - } - handle = dlopen(full_driver_name.c_str(), RTLD_NOW | RTLD_LOCAL); - if (!handle) { - error_message += "\ndlopen() failed: "; - error_message += dlerror(); + void *load_handle = nullptr; + if (entrypoint) { + status = library.Lookup(entrypoint, &load_handle, error); + } else { + auto name = AdbcDriverManagerDefaultEntrypoint(driver_name); + status = library.Lookup(name.c_str(), &load_handle, error); + if (status != ADBC_STATUS_OK) { + status = library.Lookup(kDefaultEntrypoint, &load_handle, error); } } - if (!handle) { - SetError(error, error_message); - // AdbcDatabaseInit tries to call this if set - driver->release = nullptr; - return ADBC_STATUS_INTERNAL; - } - void *load_handle = dlsym(handle, entrypoint); - if (!load_handle) { - std::string message = "dlsym("; - message += entrypoint; - message += ") failed: "; - message += dlerror(); - SetError(error, message); - return ADBC_STATUS_INTERNAL; + if (status != ADBC_STATUS_OK) { + library.Release(); + return status; } init_func = reinterpret_cast(load_handle); -#endif // defined(_WIN32) - - AdbcStatusCode status = AdbcLoadDriverFromInitFunc(init_func, version, driver, error); + status = AdbcLoadDriverFromInitFunc(init_func, version, driver, error); if (status == ADBC_STATUS_OK) { ManagerDriverState *state = new ManagerDriverState; state->driver_release = driver->release; -#if defined(_WIN32) - state->handle = handle; -#endif // defined(_WIN32) + state->handle = std::move(library); driver->release = &ReleaseDriver; driver->private_manager = state; } else { -#if defined(_WIN32) - if (!FreeLibrary(handle)) { - std::string message = "FreeLibrary() failed: "; - GetWinError(&message); - SetError(error, message); - } -#endif // defined(_WIN32) + library.Release(); } return status; } AdbcStatusCode AdbcLoadDriverFromInitFunc(AdbcDriverInitFunc init_func, int version, void *raw_driver, struct AdbcError *error) { + constexpr std::array kSupportedVersions = { + ADBC_VERSION_1_1_0, + ADBC_VERSION_1_0_0, + }; + + if (!raw_driver) { + SetError(error, "Must provide non-NULL raw_driver"); + return ADBC_STATUS_INVALID_ARGUMENT; + } + + switch (version) { + case ADBC_VERSION_1_0_0: + case ADBC_VERSION_1_1_0: + break; + default: + SetError(error, "Only ADBC 1.0.0 and 1.1.0 are supported"); + return ADBC_STATUS_NOT_IMPLEMENTED; + } + #define FILL_DEFAULT(DRIVER, STUB) \ if (!DRIVER->STUB) { \ DRIVER->STUB = &STUB; \ @@ -746,12 +1463,22 @@ AdbcStatusCode AdbcLoadDriverFromInitFunc(AdbcDriverInitFunc init_func, int vers return ADBC_STATUS_INTERNAL; \ } - auto result = init_func(version, raw_driver, error); + // Starting from the passed version, try each (older) version in + // succession with the underlying driver until we find one that's + // accepted. + AdbcStatusCode result = ADBC_STATUS_NOT_IMPLEMENTED; + for (const int try_version : kSupportedVersions) { + if (try_version > version) + continue; + result = init_func(try_version, raw_driver, error); + if (result != ADBC_STATUS_NOT_IMPLEMENTED) + break; + } if (result != ADBC_STATUS_OK) { return result; } - if (version == ADBC_VERSION_1_0_0) { + if (version >= ADBC_VERSION_1_0_0) { auto *driver = reinterpret_cast(raw_driver); CHECK_REQUIRED(driver, DatabaseNew); CHECK_REQUIRED(driver, DatabaseInit); @@ -781,6 +1508,41 @@ AdbcStatusCode AdbcLoadDriverFromInitFunc(AdbcDriverInitFunc init_func, int vers FILL_DEFAULT(driver, StatementSetSqlQuery); FILL_DEFAULT(driver, StatementSetSubstraitPlan); } + if (version >= ADBC_VERSION_1_1_0) { + auto *driver = reinterpret_cast(raw_driver); + FILL_DEFAULT(driver, ErrorGetDetailCount); + FILL_DEFAULT(driver, ErrorGetDetail); + FILL_DEFAULT(driver, ErrorFromArrayStream); + + FILL_DEFAULT(driver, DatabaseGetOption); + FILL_DEFAULT(driver, DatabaseGetOptionBytes); + FILL_DEFAULT(driver, DatabaseGetOptionDouble); + FILL_DEFAULT(driver, DatabaseGetOptionInt); + FILL_DEFAULT(driver, DatabaseSetOptionBytes); + FILL_DEFAULT(driver, DatabaseSetOptionDouble); + FILL_DEFAULT(driver, DatabaseSetOptionInt); + + FILL_DEFAULT(driver, ConnectionCancel); + FILL_DEFAULT(driver, ConnectionGetOption); + FILL_DEFAULT(driver, ConnectionGetOptionBytes); + FILL_DEFAULT(driver, ConnectionGetOptionDouble); + FILL_DEFAULT(driver, ConnectionGetOptionInt); + FILL_DEFAULT(driver, ConnectionGetStatistics); + FILL_DEFAULT(driver, ConnectionGetStatisticNames); + FILL_DEFAULT(driver, ConnectionSetOptionBytes); + FILL_DEFAULT(driver, ConnectionSetOptionDouble); + FILL_DEFAULT(driver, ConnectionSetOptionInt); + + FILL_DEFAULT(driver, StatementCancel); + FILL_DEFAULT(driver, StatementExecuteSchema); + FILL_DEFAULT(driver, StatementGetOption); + FILL_DEFAULT(driver, StatementGetOptionBytes); + FILL_DEFAULT(driver, StatementGetOptionDouble); + FILL_DEFAULT(driver, StatementGetOptionInt); + FILL_DEFAULT(driver, StatementSetOptionBytes); + FILL_DEFAULT(driver, StatementSetOptionDouble); + FILL_DEFAULT(driver, StatementSetOptionInt); + } return ADBC_STATUS_OK; diff --git a/src/duckdb/src/common/arrow/appender/list_data.cpp b/src/duckdb/src/common/arrow/appender/list_data.cpp new file mode 100644 index 000000000..50ff80687 --- /dev/null +++ b/src/duckdb/src/common/arrow/appender/list_data.cpp @@ -0,0 +1,78 @@ +#include "duckdb/common/arrow/arrow_appender.hpp" +#include "duckdb/common/arrow/appender/list_data.hpp" + +namespace duckdb { + +//===--------------------------------------------------------------------===// +// Lists +//===--------------------------------------------------------------------===// +void ArrowListData::AppendOffsets(ArrowAppendData &append_data, UnifiedVectorFormat &format, idx_t from, idx_t to, + vector &child_sel) { + // resize the offset buffer - the offset buffer holds the offsets into the child array + idx_t size = to - from; + append_data.main_buffer.resize(append_data.main_buffer.size() + sizeof(uint32_t) * (size + 1)); + auto data = UnifiedVectorFormat::GetData(format); + auto offset_data = append_data.main_buffer.GetData(); + if (append_data.row_count == 0) { + // first entry + offset_data[0] = 0; + } + // set up the offsets using the list entries + auto last_offset = offset_data[append_data.row_count]; + for (idx_t i = from; i < to; i++) { + auto source_idx = format.sel->get_index(i); + auto offset_idx = append_data.row_count + i + 1 - from; + + if (!format.validity.RowIsValid(source_idx)) { + offset_data[offset_idx] = last_offset; + continue; + } + + // append the offset data + auto list_length = data[source_idx].length; + last_offset += list_length; + offset_data[offset_idx] = last_offset; + + for (idx_t k = 0; k < list_length; k++) { + child_sel.push_back(data[source_idx].offset + k); + } + } +} + +void ArrowListData::Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity) { + auto &child_type = ListType::GetChildType(type); + result.main_buffer.reserve((capacity + 1) * sizeof(uint32_t)); + auto child_buffer = ArrowAppender::InitializeChild(child_type, capacity, result.options); + result.child_data.push_back(std::move(child_buffer)); +} + +void ArrowListData::Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { + UnifiedVectorFormat format; + input.ToUnifiedFormat(input_size, format); + idx_t size = to - from; + vector child_indices; + AppendValidity(append_data, format, from, to); + ArrowListData::AppendOffsets(append_data, format, from, to, child_indices); + + // append the child vector of the list + SelectionVector child_sel(child_indices.data()); + auto &child = ListVector::GetEntry(input); + auto child_size = child_indices.size(); + Vector child_copy(child.GetType()); + child_copy.Slice(child, child_sel, child_size); + append_data.child_data[0]->append_vector(*append_data.child_data[0], child_copy, 0, child_size, child_size); + append_data.row_count += size; +} + +void ArrowListData::Finalize(ArrowAppendData &append_data, const LogicalType &type, ArrowArray *result) { + result->n_buffers = 2; + result->buffers[1] = append_data.main_buffer.data(); + + auto &child_type = ListType::GetChildType(type); + ArrowAppender::AddChildren(append_data, 1); + result->children = append_data.child_pointers.data(); + result->n_children = 1; + append_data.child_arrays[0] = *ArrowAppender::FinalizeChild(child_type, std::move(append_data.child_data[0])); +} + +} // namespace duckdb diff --git a/src/duckdb/src/common/arrow/appender/map_data.cpp b/src/duckdb/src/common/arrow/appender/map_data.cpp new file mode 100644 index 000000000..3bacf653c --- /dev/null +++ b/src/duckdb/src/common/arrow/appender/map_data.cpp @@ -0,0 +1,91 @@ +#include "duckdb/common/arrow/arrow_appender.hpp" +#include "duckdb/common/arrow/appender/map_data.hpp" +#include "duckdb/common/arrow/appender/list_data.hpp" + +namespace duckdb { + +//===--------------------------------------------------------------------===// +// Maps +//===--------------------------------------------------------------------===// +void ArrowMapData::Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity) { + // map types are stored in a (too) clever way + // the main buffer holds the null values and the offsets + // then we have a single child, which is a struct of the map_type, and the key_type + result.main_buffer.reserve((capacity + 1) * sizeof(uint32_t)); + + auto &key_type = MapType::KeyType(type); + auto &value_type = MapType::ValueType(type); + auto internal_struct = make_uniq(result.options); + internal_struct->child_data.push_back(ArrowAppender::InitializeChild(key_type, capacity, result.options)); + internal_struct->child_data.push_back(ArrowAppender::InitializeChild(value_type, capacity, result.options)); + + result.child_data.push_back(std::move(internal_struct)); +} + +void ArrowMapData::Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { + UnifiedVectorFormat format; + input.ToUnifiedFormat(input_size, format); + idx_t size = to - from; + AppendValidity(append_data, format, from, to); + vector child_indices; + ArrowListData::AppendOffsets(append_data, format, from, to, child_indices); + + SelectionVector child_sel(child_indices.data()); + auto &key_vector = MapVector::GetKeys(input); + auto &value_vector = MapVector::GetValues(input); + auto list_size = child_indices.size(); + + auto &struct_data = *append_data.child_data[0]; + auto &key_data = *struct_data.child_data[0]; + auto &value_data = *struct_data.child_data[1]; + + Vector key_vector_copy(key_vector.GetType()); + key_vector_copy.Slice(key_vector, child_sel, list_size); + Vector value_vector_copy(value_vector.GetType()); + value_vector_copy.Slice(value_vector, child_sel, list_size); + key_data.append_vector(key_data, key_vector_copy, 0, list_size, list_size); + value_data.append_vector(value_data, value_vector_copy, 0, list_size, list_size); + + append_data.row_count += size; + struct_data.row_count += size; +} + +void ArrowMapData::Finalize(ArrowAppendData &append_data, const LogicalType &type, ArrowArray *result) { + // set up the main map buffer + D_ASSERT(result); + result->n_buffers = 2; + result->buffers[1] = append_data.main_buffer.data(); + + // the main map buffer has a single child: a struct + ArrowAppender::AddChildren(append_data, 1); + result->children = append_data.child_pointers.data(); + result->n_children = 1; + + auto &struct_data = *append_data.child_data[0]; + auto struct_result = ArrowAppender::FinalizeChild(type, std::move(append_data.child_data[0])); + + // Initialize the struct array data + const auto struct_child_count = 2; + ArrowAppender::AddChildren(struct_data, struct_child_count); + struct_result->children = struct_data.child_pointers.data(); + struct_result->n_buffers = 1; + struct_result->n_children = struct_child_count; + struct_result->length = struct_data.child_data[0]->row_count; + + append_data.child_arrays[0] = *struct_result; + + D_ASSERT(struct_data.child_data[0]->row_count == struct_data.child_data[1]->row_count); + + auto &key_type = MapType::KeyType(type); + auto &value_type = MapType::ValueType(type); + auto key_data = ArrowAppender::FinalizeChild(key_type, std::move(struct_data.child_data[0])); + struct_data.child_arrays[0] = *key_data; + struct_data.child_arrays[1] = *ArrowAppender::FinalizeChild(value_type, std::move(struct_data.child_data[1])); + + // keys cannot have null values + if (key_data->null_count > 0) { + throw std::runtime_error("Arrow doesn't accept NULL keys on Maps"); + } +} + +} // namespace duckdb diff --git a/src/duckdb/src/common/arrow/arrow_appender.cpp b/src/duckdb/src/common/arrow/arrow_appender.cpp index bbca77756..10d1e39e3 100644 --- a/src/duckdb/src/common/arrow/arrow_appender.cpp +++ b/src/duckdb/src/common/arrow/arrow_appender.cpp @@ -193,26 +193,26 @@ static void InitializeFunctionPointers(ArrowAppendData &append_data, const Logic if (append_data.options.arrow_offset_size == ArrowOffsetSize::LARGE) { InitializeAppenderForType>(append_data); } else { - InitializeAppenderForType>(append_data); + InitializeAppenderForType>(append_data); } break; case LogicalTypeId::UUID: if (append_data.options.arrow_offset_size == ArrowOffsetSize::LARGE) { InitializeAppenderForType>(append_data); } else { - InitializeAppenderForType>(append_data); + InitializeAppenderForType>(append_data); } break; case LogicalTypeId::ENUM: switch (type.InternalType()) { case PhysicalType::UINT8: - InitializeAppenderForType>(append_data); + InitializeAppenderForType>(append_data); break; case PhysicalType::UINT16: - InitializeAppenderForType>(append_data); + InitializeAppenderForType>(append_data); break; case PhysicalType::UINT32: - InitializeAppenderForType>(append_data); + InitializeAppenderForType>(append_data); break; default: throw InternalException("Unsupported internal enum type"); @@ -227,20 +227,11 @@ static void InitializeFunctionPointers(ArrowAppendData &append_data, const Logic case LogicalTypeId::STRUCT: InitializeAppenderForType(append_data); break; - case LogicalTypeId::LIST: { - if (append_data.options.arrow_offset_size == ArrowOffsetSize::LARGE) { - InitializeAppenderForType>(append_data); - } else { - InitializeAppenderForType>(append_data); - } + case LogicalTypeId::LIST: + InitializeAppenderForType(append_data); break; - } case LogicalTypeId::MAP: - if (append_data.options.arrow_offset_size == ArrowOffsetSize::LARGE) { - InitializeAppenderForType>(append_data); - } else { - InitializeAppenderForType>(append_data); - } + InitializeAppenderForType(append_data); break; default: throw NotImplementedException("Unsupported type in DuckDB -> Arrow Conversion: %s\n", type.ToString()); diff --git a/src/duckdb/src/common/arrow/arrow_converter.cpp b/src/duckdb/src/common/arrow/arrow_converter.cpp index d57bcc471..0ecc46e0f 100644 --- a/src/duckdb/src/common/arrow/arrow_converter.cpp +++ b/src/duckdb/src/common/arrow/arrow_converter.cpp @@ -187,11 +187,7 @@ void SetArrowFormat(DuckDBArrowSchemaHolder &root_holder, ArrowSchema &child, co break; } case LogicalTypeId::LIST: { - if (options.arrow_offset_size == ArrowOffsetSize::LARGE) { - child.format = "+L"; - } else { - child.format = "+l"; - } + child.format = "+l"; child.n_children = 1; root_holder.nested_children.emplace_back(); root_holder.nested_children.back().resize(1); diff --git a/src/duckdb/src/common/enum_util.cpp b/src/duckdb/src/common/enum_util.cpp index 64fa25994..f1c62822d 100644 --- a/src/duckdb/src/common/enum_util.cpp +++ b/src/duckdb/src/common/enum_util.cpp @@ -60,11 +60,11 @@ #include "duckdb/common/types/conflict_manager.hpp" #include "duckdb/common/types/hyperloglog.hpp" #include "duckdb/common/types/row/partitioned_tuple_data.hpp" +#include "duckdb/common/types/row/tuple_data_collection.hpp" #include "duckdb/common/types/row/tuple_data_states.hpp" #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/types/vector.hpp" #include "duckdb/common/types/vector_buffer.hpp" -#include "duckdb/core_functions/aggregate/quantile_enum.hpp" #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/node.hpp" #include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" @@ -1558,6 +1558,8 @@ const char* EnumUtil::ToChars(ExpressionClass value) { return "POSITIONAL_REFERENCE"; case ExpressionClass::BETWEEN: return "BETWEEN"; + case ExpressionClass::LAMBDA_REF: + return "LAMBDA_REF"; case ExpressionClass::BOUND_AGGREGATE: return "BOUND_AGGREGATE"; case ExpressionClass::BOUND_CASE: @@ -1660,6 +1662,9 @@ ExpressionClass EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "BETWEEN")) { return ExpressionClass::BETWEEN; } + if (StringUtil::Equals(value, "LAMBDA_REF")) { + return ExpressionClass::LAMBDA_REF; + } if (StringUtil::Equals(value, "BOUND_AGGREGATE")) { return ExpressionClass::BOUND_AGGREGATE; } @@ -1841,6 +1846,8 @@ const char* EnumUtil::ToChars(ExpressionType value) { return "FUNCTION_REF"; case ExpressionType::TABLE_REF: return "TABLE_REF"; + case ExpressionType::LAMBDA_REF: + return "LAMBDA_REF"; case ExpressionType::CAST: return "CAST"; case ExpressionType::BOUND_REF: @@ -2041,6 +2048,9 @@ ExpressionType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "TABLE_REF")) { return ExpressionType::TABLE_REF; } + if (StringUtil::Equals(value, "LAMBDA_REF")) { + return ExpressionType::LAMBDA_REF; + } if (StringUtil::Equals(value, "CAST")) { return ExpressionType::CAST; } @@ -2117,6 +2127,8 @@ const char* EnumUtil::ToChars(ExtraTypeInfoType value) { return "USER_TYPE_INFO"; case ExtraTypeInfoType::AGGREGATE_STATE_TYPE_INFO: return "AGGREGATE_STATE_TYPE_INFO"; + case ExtraTypeInfoType::ARRAY_TYPE_INFO: + return "ARRAY_TYPE_INFO"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -2151,6 +2163,9 @@ ExtraTypeInfoType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "AGGREGATE_STATE_TYPE_INFO")) { return ExtraTypeInfoType::AGGREGATE_STATE_TYPE_INFO; } + if (StringUtil::Equals(value, "ARRAY_TYPE_INFO")) { + return ExtraTypeInfoType::ARRAY_TYPE_INFO; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -3070,6 +3085,8 @@ const char* EnumUtil::ToChars(LogicalTypeId value) { return "LAMBDA"; case LogicalTypeId::UNION: return "UNION"; + case LogicalTypeId::ARRAY: + return "ARRAY"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -3203,6 +3220,9 @@ LogicalTypeId EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "UNION")) { return LogicalTypeId::UNION; } + if (StringUtil::Equals(value, "ARRAY")) { + return LogicalTypeId::ARRAY; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -4411,6 +4431,8 @@ const char* EnumUtil::ToChars(PhysicalType value) { return "LIST"; case PhysicalType::STRUCT: return "STRUCT"; + case PhysicalType::ARRAY: + return "ARRAY"; case PhysicalType::VARCHAR: return "VARCHAR"; case PhysicalType::INT128: @@ -4470,6 +4492,9 @@ PhysicalType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "STRUCT")) { return PhysicalType::STRUCT; } + if (StringUtil::Equals(value, "ARRAY")) { + return PhysicalType::ARRAY; + } if (StringUtil::Equals(value, "VARCHAR")) { return PhysicalType::VARCHAR; } @@ -4572,44 +4597,6 @@ ProfilerPrintFormat EnumUtil::FromString(const char *value) throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } -template<> -const char* EnumUtil::ToChars(QuantileSerializationType value) { - switch(value) { - case QuantileSerializationType::NON_DECIMAL: - return "NON_DECIMAL"; - case QuantileSerializationType::DECIMAL_DISCRETE: - return "DECIMAL_DISCRETE"; - case QuantileSerializationType::DECIMAL_DISCRETE_LIST: - return "DECIMAL_DISCRETE_LIST"; - case QuantileSerializationType::DECIMAL_CONTINUOUS: - return "DECIMAL_CONTINUOUS"; - case QuantileSerializationType::DECIMAL_CONTINUOUS_LIST: - return "DECIMAL_CONTINUOUS_LIST"; - default: - throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); - } -} - -template<> -QuantileSerializationType EnumUtil::FromString(const char *value) { - if (StringUtil::Equals(value, "NON_DECIMAL")) { - return QuantileSerializationType::NON_DECIMAL; - } - if (StringUtil::Equals(value, "DECIMAL_DISCRETE")) { - return QuantileSerializationType::DECIMAL_DISCRETE; - } - if (StringUtil::Equals(value, "DECIMAL_DISCRETE_LIST")) { - return QuantileSerializationType::DECIMAL_DISCRETE_LIST; - } - if (StringUtil::Equals(value, "DECIMAL_CONTINUOUS")) { - return QuantileSerializationType::DECIMAL_CONTINUOUS; - } - if (StringUtil::Equals(value, "DECIMAL_CONTINUOUS_LIST")) { - return QuantileSerializationType::DECIMAL_CONTINUOUS_LIST; - } - throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); -} - template<> const char* EnumUtil::ToChars(QueryNodeType value) { switch(value) { @@ -5157,29 +5144,6 @@ SinkFinalizeType EnumUtil::FromString(const char *value) { throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } -template<> -const char* EnumUtil::ToChars(SinkNextBatchType value) { - switch(value) { - case SinkNextBatchType::READY: - return "READY"; - case SinkNextBatchType::BLOCKED: - return "BLOCKED"; - default: - throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); - } -} - -template<> -SinkNextBatchType EnumUtil::FromString(const char *value) { - if (StringUtil::Equals(value, "READY")) { - return SinkNextBatchType::READY; - } - if (StringUtil::Equals(value, "BLOCKED")) { - return SinkNextBatchType::BLOCKED; - } - throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); -} - template<> const char* EnumUtil::ToChars(SinkResultType value) { switch(value) { @@ -5435,6 +5399,8 @@ const char* EnumUtil::ToChars(StatisticsType value) { return "STRUCT_STATS"; case StatisticsType::BASE_STATS: return "BASE_STATS"; + case StatisticsType::ARRAY_STATS: + return "ARRAY_STATS"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -5457,6 +5423,9 @@ StatisticsType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "BASE_STATS")) { return StatisticsType::BASE_STATS; } + if (StringUtil::Equals(value, "ARRAY_STATS")) { + return StatisticsType::ARRAY_STATS; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -6141,6 +6110,8 @@ const char* EnumUtil::ToChars(VectorBufferType value) { return "MANAGED_BUFFER"; case VectorBufferType::OPAQUE_BUFFER: return "OPAQUE_BUFFER"; + case VectorBufferType::ARRAY_BUFFER: + return "ARRAY_BUFFER"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -6175,6 +6146,9 @@ VectorBufferType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "OPAQUE_BUFFER")) { return VectorBufferType::OPAQUE_BUFFER; } + if (StringUtil::Equals(value, "ARRAY_BUFFER")) { + return VectorBufferType::ARRAY_BUFFER; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -6526,5 +6500,66 @@ WindowBoundary EnumUtil::FromString(const char *value) { throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(WindowExcludeMode value) { + switch(value) { + case WindowExcludeMode::NO_OTHER: + return "NO_OTHER"; + case WindowExcludeMode::CURRENT_ROW: + return "CURRENT_ROW"; + case WindowExcludeMode::GROUP: + return "GROUP"; + case WindowExcludeMode::TIES: + return "TIES"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +WindowExcludeMode EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "NO_OTHER")) { + return WindowExcludeMode::NO_OTHER; + } + if (StringUtil::Equals(value, "CURRENT_ROW")) { + return WindowExcludeMode::CURRENT_ROW; + } + if (StringUtil::Equals(value, "GROUP")) { + return WindowExcludeMode::GROUP; + } + if (StringUtil::Equals(value, "TIES")) { + return WindowExcludeMode::TIES; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + +template<> +const char* EnumUtil::ToChars(WithinCollection value) { + switch(value) { + case WithinCollection::NO: + return "NO"; + case WithinCollection::LIST: + return "LIST"; + case WithinCollection::ARRAY: + return "ARRAY"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +WithinCollection EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "NO")) { + return WithinCollection::NO; + } + if (StringUtil::Equals(value, "LIST")) { + return WithinCollection::LIST; + } + if (StringUtil::Equals(value, "ARRAY")) { + return WithinCollection::ARRAY; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + } diff --git a/src/duckdb/src/common/enums/expression_type.cpp b/src/duckdb/src/common/enums/expression_type.cpp index f6755c3cb..c017b5059 100644 --- a/src/duckdb/src/common/enums/expression_type.cpp +++ b/src/duckdb/src/common/enums/expression_type.cpp @@ -99,6 +99,8 @@ string ExpressionTypeToString(ExpressionType type) { return "PLACEHOLDER"; case ExpressionType::COLUMN_REF: return "COLUMN_REF"; + case ExpressionType::LAMBDA_REF: + return "LAMBDA_REF"; case ExpressionType::FUNCTION_REF: return "FUNCTION_REF"; case ExpressionType::TABLE_REF: @@ -156,6 +158,8 @@ string ExpressionClassToString(ExpressionClass type) { return "CAST"; case ExpressionClass::COLUMN_REF: return "COLUMN_REF"; + case ExpressionClass::LAMBDA_REF: + return "LAMBDA_REF"; case ExpressionClass::COMPARISON: return "COMPARISON"; case ExpressionClass::CONJUNCTION: diff --git a/src/duckdb/src/common/extra_type_info.cpp b/src/duckdb/src/common/extra_type_info.cpp index 235b5b7cd..e00262465 100644 --- a/src/duckdb/src/common/extra_type_info.cpp +++ b/src/duckdb/src/common/extra_type_info.cpp @@ -312,4 +312,17 @@ void EnumTypeInfo::Serialize(Serializer &serializer) const { [&](Serializer::List &list, idx_t i) { list.WriteElement(strings[i]); }); } +//===--------------------------------------------------------------------===// +// ArrayTypeInfo +//===--------------------------------------------------------------------===// + +ArrayTypeInfo::ArrayTypeInfo(LogicalType child_type_p, idx_t size_p) + : ExtraTypeInfo(ExtraTypeInfoType::ARRAY_TYPE_INFO), child_type(std::move(child_type_p)), size(size_p) { +} + +bool ArrayTypeInfo::EqualsInternal(ExtraTypeInfo *other_p) const { + auto &other = other_p->Cast(); + return child_type == other.child_type && size == other.size; +} + } // namespace duckdb diff --git a/src/duckdb/src/common/file_system.cpp b/src/duckdb/src/common/file_system.cpp index ed5a4c843..e73779a7d 100644 --- a/src/duckdb/src/common/file_system.cpp +++ b/src/duckdb/src/common/file_system.cpp @@ -344,7 +344,7 @@ bool FileSystem::FileExists(const string &filename) { } bool FileSystem::IsPipe(const string &filename) { - return false; + throw NotImplementedException("%s: IsPipe is not implemented!", GetName()); } void FileSystem::RemoveFile(const string &filename) { @@ -500,10 +500,6 @@ bool FileHandle::CanSeek() { return file_system.CanSeek(); } -bool FileHandle::IsPipe() { - return file_system.IsPipe(path); -} - string FileHandle::ReadLine() { string result; char buffer[1]; @@ -539,7 +535,7 @@ FileType FileHandle::GetType() { } bool FileSystem::IsRemoteFile(const string &path) { - const string prefixes[] = {"http://", "https://", "s3://"}; + const string prefixes[] = {"http://", "https://", "s3://", "gcs://", "r2://"}; for (auto &prefix : prefixes) { if (StringUtil::StartsWith(path, prefix)) { return true; diff --git a/src/duckdb/src/common/multi_file_reader.cpp b/src/duckdb/src/common/multi_file_reader.cpp index 9049008e3..5f28fa026 100644 --- a/src/duckdb/src/common/multi_file_reader.cpp +++ b/src/duckdb/src/common/multi_file_reader.cpp @@ -1,12 +1,15 @@ #include "duckdb/common/multi_file_reader.hpp" -#include "duckdb/function/table_function.hpp" -#include "duckdb/main/config.hpp" -#include "duckdb/common/types/value.hpp" -#include "duckdb/planner/operator/logical_get.hpp" + #include "duckdb/common/exception.hpp" -#include "duckdb/function/function_set.hpp" #include "duckdb/common/hive_partitioning.hpp" #include "duckdb/common/types.hpp" +#include "duckdb/common/types/value.hpp" +#include "duckdb/function/function_set.hpp" +#include "duckdb/function/table_function.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/planner/operator/logical_get.hpp" + +#include namespace duckdb { @@ -32,6 +35,10 @@ vector MultiFileReader::GetFileList(ClientContext &context, const Value if (input.type().id() == LogicalTypeId::VARCHAR) { auto file_name = StringValue::Get(input); files = fs.GlobFiles(file_name, context, options); + + // Sort the files to ensure that the order is deterministic + std::sort(files.begin(), files.end()); + } else if (input.type().id() == LogicalTypeId::LIST) { for (auto &val : ListValue::GetChildren(input)) { if (val.IsNull()) { @@ -41,6 +48,7 @@ vector MultiFileReader::GetFileList(ClientContext &context, const Value throw ParserException("%s reader can only take a list of strings as a parameter", name); } auto glob_files = fs.GlobFiles(StringValue::Get(val), context, options); + std::sort(glob_files.begin(), glob_files.end()); files.insert(files.end(), glob_files.begin(), glob_files.end()); } } else { @@ -49,6 +57,7 @@ vector MultiFileReader::GetFileList(ClientContext &context, const Value if (files.empty() && options == FileGlobOptions::DISALLOW_EMPTY) { throw IOException("%s reader needs at least one file to read", name); } + return files; } @@ -308,6 +317,11 @@ void MultiFileReader::CreateMapping(const string &file_name, const vector &global_types, optional_ptr filters, + MultiFileReaderData &reader_data) { if (filters) { reader_data.filter_map.resize(global_types.size()); for (idx_t c = 0; c < reader_data.column_mapping.size(); c++) { @@ -434,7 +448,7 @@ void MultiFileReaderOptions::AutoDetectHiveTypesInternal(const string &file, Cli } Value value(part.second); for (auto &candidate : candidates) { - const bool success = value.TryCastAs(context, candidate, true); + const bool success = value.TryCastAs(context, candidate); if (success) { hive_types_schema[name] = candidate; break; diff --git a/src/duckdb/src/common/operator/cast_operators.cpp b/src/duckdb/src/common/operator/cast_operators.cpp index 8de8cd5b1..3877a1dca 100644 --- a/src/duckdb/src/common/operator/cast_operators.cpp +++ b/src/duckdb/src/common/operator/cast_operators.cpp @@ -1368,11 +1368,17 @@ timestamp_t CastTimestampUsToSec::Operation(timestamp_t input) { timestamp_t cast_timestamp(Timestamp::GetEpochSeconds(input)); return cast_timestamp; } + template <> timestamp_t CastTimestampMsToUs::Operation(timestamp_t input) { return Timestamp::FromEpochMs(input.value); } +template <> +date_t CastTimestampMsToDate::Operation(timestamp_t input) { + return Timestamp::GetDate(Timestamp::FromEpochMs(input.value)); +} + template <> timestamp_t CastTimestampMsToNs::Operation(timestamp_t input) { auto us = CastTimestampMsToUs::Operation(input); @@ -1385,8 +1391,9 @@ timestamp_t CastTimestampNsToUs::Operation(timestamp_t input) { } template <> -timestamp_t CastTimestampSecToUs::Operation(timestamp_t input) { - return Timestamp::FromEpochSeconds(input.value); +date_t CastTimestampNsToDate::Operation(timestamp_t input) { + const auto us = CastTimestampNsToUs::Operation(input); + return Timestamp::GetDate(us); } template <> @@ -1395,12 +1402,23 @@ timestamp_t CastTimestampSecToMs::Operation(timestamp_t input) { return CastTimestampUsToMs::Operation(us); } +template <> +timestamp_t CastTimestampSecToUs::Operation(timestamp_t input) { + return Timestamp::FromEpochSeconds(input.value); +} + template <> timestamp_t CastTimestampSecToNs::Operation(timestamp_t input) { auto us = CastTimestampSecToUs::Operation(input); return CastTimestampUsToNs::Operation(us); } +template <> +date_t CastTimestampSecToDate::Operation(timestamp_t input) { + const auto us = CastTimestampSecToUs::Operation(input); + return Timestamp::GetDate(us); +} + //===--------------------------------------------------------------------===// // Cast To Timestamp //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/common/row_operations/row_gather.cpp b/src/duckdb/src/common/row_operations/row_gather.cpp index 631c5ffa6..c04529024 100644 --- a/src/duckdb/src/common/row_operations/row_gather.cpp +++ b/src/duckdb/src/common/row_operations/row_gather.cpp @@ -165,6 +165,7 @@ void RowOperations::Gather(Vector &rows, const SelectionVector &row_sel, Vector break; case PhysicalType::LIST: case PhysicalType::STRUCT: + case PhysicalType::ARRAY: GatherNestedVector(rows, row_sel, col, col_sel, count, layout, col_no, heap_ptr); break; default: diff --git a/src/duckdb/src/common/row_operations/row_heap_gather.cpp b/src/duckdb/src/common/row_operations/row_heap_gather.cpp index a8b6e7b9f..b709f054b 100644 --- a/src/duckdb/src/common/row_operations/row_heap_gather.cpp +++ b/src/duckdb/src/common/row_operations/row_heap_gather.cpp @@ -133,6 +133,88 @@ static void HeapGatherListVector(Vector &v, const idx_t vcount, const SelectionV } } +static void HeapGatherArrayVector(Vector &v, const idx_t vcount, const SelectionVector &sel, + data_ptr_t *key_locations) { + // Setup + auto &validity = FlatVector::Validity(v); + auto &child_type = ArrayType::GetChildType(v.GetType()); + auto array_size = ArrayType::GetSize(v.GetType()); + auto &child_vector = ArrayVector::GetEntry(v); + auto child_type_size = GetTypeIdSize(child_type.InternalType()); + auto child_type_is_var_size = !TypeIsConstantSize(child_type.InternalType()); + + data_ptr_t array_entry_locations[STANDARD_VECTOR_SIZE]; + + // array must have a validitymask for its elements + auto array_validitymask_size = (array_size + 7) / 8; + + auto &child_validity = FlatVector::Validity(child_vector); + + for (idx_t i = 0; i < vcount; i++) { + // row idx + const auto col_idx = sel.get_index(i); + if (!validity.RowIsValid(col_idx)) { + // we still need to zero out the child validity corresponding to this row + for (idx_t elem_idx = 0; elem_idx < array_size; elem_idx++) { + child_validity.Set(col_idx * array_size + elem_idx, false); + } + continue; + } + + // Setup validity mask + data_ptr_t array_validitymask_location = key_locations[i]; + key_locations[i] += array_validitymask_size; + + // The size of each variable size entry is stored after the validity mask + // (if the child type is variable size) + data_ptr_t var_entry_size_ptr = nullptr; + if (child_type_is_var_size) { + var_entry_size_ptr = key_locations[i]; + key_locations[i] += array_size * sizeof(idx_t); + } + + auto array_start = col_idx * array_size; + auto elem_remaining = array_size; + + idx_t offset_in_byte = 0; + + while (elem_remaining > 0) { + auto chunk_size = MinValue(static_cast(STANDARD_VECTOR_SIZE), elem_remaining); + for (idx_t elem_idx = 0; elem_idx < chunk_size; elem_idx++) { + child_validity.Set(array_start + elem_idx, *(array_validitymask_location) & (1 << offset_in_byte)); + if (++offset_in_byte == 8) { + array_validitymask_location++; + offset_in_byte = 0; + } + } + + SelectionVector array_sel(STANDARD_VECTOR_SIZE); + + if (child_type_is_var_size) { + // variable size list entries + for (idx_t elem_idx = 0; elem_idx < chunk_size; elem_idx++) { + array_entry_locations[elem_idx] = key_locations[i]; + key_locations[i] += Load(var_entry_size_ptr); + var_entry_size_ptr += sizeof(idx_t); + array_sel.set_index(elem_idx, array_start + elem_idx); + } + } else { + // constant size list entries + for (idx_t elem_idx = 0; elem_idx < chunk_size; elem_idx++) { + array_entry_locations[elem_idx] = key_locations[i]; + key_locations[i] += child_type_size; + array_sel.set_index(elem_idx, array_start + elem_idx); + } + } + + RowOperations::HeapGather(child_vector, chunk_size, array_sel, 0, array_entry_locations, nullptr); + + elem_remaining -= chunk_size; + array_start += chunk_size; + } + } +} + void RowOperations::HeapGather(Vector &v, const idx_t &vcount, const SelectionVector &sel, const idx_t &col_no, data_ptr_t *key_locations, data_ptr_t *validitymask_locations) { v.SetVectorType(VectorType::FLAT_VECTOR); @@ -200,6 +282,9 @@ void RowOperations::HeapGather(Vector &v, const idx_t &vcount, const SelectionVe case PhysicalType::LIST: HeapGatherListVector(v, vcount, sel, key_locations); break; + case PhysicalType::ARRAY: + HeapGatherArrayVector(v, vcount, sel, key_locations); + break; default: throw NotImplementedException("Unimplemented deserialize from row-format"); } diff --git a/src/duckdb/src/common/row_operations/row_heap_scatter.cpp b/src/duckdb/src/common/row_operations/row_heap_scatter.cpp index c51e2b5d9..6665442f3 100644 --- a/src/duckdb/src/common/row_operations/row_heap_scatter.cpp +++ b/src/duckdb/src/common/row_operations/row_heap_scatter.cpp @@ -78,6 +78,48 @@ static void ComputeListEntrySizes(Vector &v, UnifiedVectorFormat &vdata, idx_t e } } +static void ComputeArrayEntrySizes(Vector &v, UnifiedVectorFormat &vdata, idx_t entry_sizes[], idx_t ser_count, + const SelectionVector &sel, idx_t offset) { + + auto array_size = ArrayType::GetSize(v.GetType()); + auto child_vector = ArrayVector::GetEntry(v); + + idx_t array_entry_sizes[STANDARD_VECTOR_SIZE]; + + for (idx_t i = 0; i < ser_count; i++) { + + // Validity for the array elements + entry_sizes[i] += (array_size + 7) / 8; + + // serialize size of each entry (if non-constant size) + if (!TypeIsConstantSize(ArrayType::GetChildType(v.GetType()).InternalType())) { + entry_sizes[i] += array_size * sizeof(idx_t); + } + + auto elem_idx = sel.get_index(i); + auto source_idx = vdata.sel->get_index(elem_idx + offset); + + auto array_start = source_idx * array_size; + auto elem_remaining = array_size; + + // the array could span multiple vectors, so we divide it into chunks + while (elem_remaining > 0) { + auto chunk_size = MinValue(static_cast(STANDARD_VECTOR_SIZE), elem_remaining); + + // compute and add to the total + std::fill_n(array_entry_sizes, chunk_size, 0); + RowOperations::ComputeEntrySizes(child_vector, array_entry_sizes, chunk_size, chunk_size, + *FlatVector::IncrementalSelectionVector(), array_start); + for (idx_t arr_elem_idx = 0; arr_elem_idx < chunk_size; arr_elem_idx++) { + entry_sizes[i] += array_entry_sizes[arr_elem_idx]; + } + // update for next iteration + elem_remaining -= chunk_size; + array_start += chunk_size; + } + } +} + void RowOperations::ComputeEntrySizes(Vector &v, UnifiedVectorFormat &vdata, idx_t entry_sizes[], idx_t vcount, idx_t ser_count, const SelectionVector &sel, idx_t offset) { const auto physical_type = v.GetType().InternalType(); @@ -97,6 +139,9 @@ void RowOperations::ComputeEntrySizes(Vector &v, UnifiedVectorFormat &vdata, idx case PhysicalType::LIST: ComputeListEntrySizes(v, vdata, entry_sizes, ser_count, sel, offset); break; + case PhysicalType::ARRAY: + ComputeArrayEntrySizes(v, vdata, entry_sizes, ser_count, sel, offset); + break; default: // LCOV_EXCL_START throw NotImplementedException("Column with variable size type %s cannot be serialized to row-format", @@ -240,7 +285,6 @@ static void HeapScatterListVector(Vector &v, idx_t vcount, const SelectionVector ValidityBytes::GetEntryIndex(col_no, entry_idx, idx_in_entry); auto list_data = ListVector::GetData(v); - auto &child_vector = ListVector::GetEntry(v); UnifiedVectorFormat list_vdata; @@ -331,6 +375,108 @@ static void HeapScatterListVector(Vector &v, idx_t vcount, const SelectionVector } } +static void HeapScatterArrayVector(Vector &v, idx_t vcount, const SelectionVector &sel, idx_t ser_count, idx_t col_idx, + data_ptr_t *key_locations, data_ptr_t *validitymask_locations, idx_t offset) { + + auto &child_vector = ArrayVector::GetEntry(v); + auto array_size = ArrayType::GetSize(v.GetType()); + auto child_type = ArrayType::GetChildType(v.GetType()); + auto child_type_size = GetTypeIdSize(child_type.InternalType()); + auto child_type_is_var_size = !TypeIsConstantSize(child_type.InternalType()); + + UnifiedVectorFormat vdata; + v.ToUnifiedFormat(vcount, vdata); + + UnifiedVectorFormat child_vdata; + child_vector.ToUnifiedFormat(ArrayVector::GetTotalSize(v), child_vdata); + + data_ptr_t array_entry_locations[STANDARD_VECTOR_SIZE]; + idx_t array_entry_sizes[STANDARD_VECTOR_SIZE]; + + // array must have a validitymask for its elements + auto array_validitymask_size = (array_size + 7) / 8; + + idx_t entry_idx; + idx_t idx_in_entry; + ValidityBytes::GetEntryIndex(col_idx, entry_idx, idx_in_entry); + const auto bit = ~(1UL << idx_in_entry); + + for (idx_t i = 0; i < ser_count; i++) { + auto source_idx = vdata.sel->get_index(sel.get_index(i) + offset); + + // First off, set the validity of the mask itself in the parent entry + if (validitymask_locations && !vdata.validity.RowIsValid(source_idx)) { + *(validitymask_locations[i] + entry_idx) &= bit; + } + + // Now we can serialize the array itself + // Every array starts with a validity mask for the children + data_ptr_t array_validitymask_location = key_locations[i]; + memset(array_validitymask_location, -1, (array_size + 7) / 8); + key_locations[i] += array_validitymask_size; + + // If the array contains variable size entries, we reserve spaces for them here + data_ptr_t var_entry_size_ptr = nullptr; + if (child_type_is_var_size) { + var_entry_size_ptr = key_locations[i]; + key_locations[i] += array_size * sizeof(idx_t); + } + + // Then comes the elements + auto array_start = source_idx * array_size; + auto elem_remaining = array_size; + + idx_t offset_in_byte = 0; + + while (elem_remaining > 0) { + // the array elements can span multiple vectors, so we divide it into chunks + auto chunk_size = MinValue(static_cast(STANDARD_VECTOR_SIZE), elem_remaining); + + // serialize list validity + for (idx_t elem_idx = 0; elem_idx < chunk_size; elem_idx++) { + auto idx_in_array = child_vdata.sel->get_index(array_start + elem_idx); + if (!child_vdata.validity.RowIsValid(idx_in_array)) { + *(array_validitymask_location) &= ~(1UL << offset_in_byte); + } + if (++offset_in_byte == 8) { + array_validitymask_location++; + offset_in_byte = 0; + } + } + + // Setup the locations for the elements + if (child_type_is_var_size) { + // The elements are variable sized + std::fill_n(array_entry_sizes, chunk_size, 0); + RowOperations::ComputeEntrySizes(child_vector, array_entry_sizes, chunk_size, chunk_size, + *FlatVector::IncrementalSelectionVector(), array_start); + for (idx_t elem_idx = 0; elem_idx < chunk_size; elem_idx++) { + array_entry_locations[elem_idx] = key_locations[i]; + key_locations[i] += array_entry_sizes[elem_idx]; + + // Now store the size of the entry + Store(array_entry_sizes[elem_idx], var_entry_size_ptr); + var_entry_size_ptr += sizeof(idx_t); + } + } else { + // The elements are constant sized + for (idx_t elem_idx = 0; elem_idx < chunk_size; elem_idx++) { + array_entry_locations[elem_idx] = key_locations[i]; + key_locations[i] += child_type_size; + } + } + + RowOperations::HeapScatter(child_vector, ArrayVector::GetTotalSize(v), + *FlatVector::IncrementalSelectionVector(), chunk_size, 0, array_entry_locations, + nullptr, array_start); + + // update for next iteration + elem_remaining -= chunk_size; + array_start += chunk_size; + } + } +} + void RowOperations::HeapScatter(Vector &v, idx_t vcount, const SelectionVector &sel, idx_t ser_count, idx_t col_idx, data_ptr_t *key_locations, data_ptr_t *validitymask_locations, idx_t offset) { if (TypeIsConstantSize(v.GetType().InternalType())) { @@ -349,6 +495,9 @@ void RowOperations::HeapScatter(Vector &v, idx_t vcount, const SelectionVector & case PhysicalType::LIST: HeapScatterListVector(v, vcount, sel, ser_count, col_idx, key_locations, validitymask_locations, offset); break; + case PhysicalType::ARRAY: + HeapScatterArrayVector(v, vcount, sel, ser_count, col_idx, key_locations, validitymask_locations, offset); + break; default: // LCOV_EXCL_START throw NotImplementedException("Serialization of variable length vector with type %s", diff --git a/src/duckdb/src/common/row_operations/row_matcher.cpp b/src/duckdb/src/common/row_operations/row_matcher.cpp index 958670963..cca637209 100644 --- a/src/duckdb/src/common/row_operations/row_matcher.cpp +++ b/src/duckdb/src/common/row_operations/row_matcher.cpp @@ -250,6 +250,9 @@ MatchFunction RowMatcher::GetMatchFunction(const LogicalType &type, const Expres return GetStructMatchFunction(type, predicate); case PhysicalType::LIST: return GetListMatchFunction(predicate); + case PhysicalType::ARRAY: + // Same logic as for lists + return GetListMatchFunction(predicate); default: throw InternalException("Unsupported PhysicalType for RowMatcher::GetMatchFunction: %s", EnumUtil::ToString(type.InternalType())); diff --git a/src/duckdb/src/common/row_operations/row_radix_scatter.cpp b/src/duckdb/src/common/row_operations/row_radix_scatter.cpp index 1a8acaec8..fc4ef8f54 100644 --- a/src/duckdb/src/common/row_operations/row_radix_scatter.cpp +++ b/src/duckdb/src/common/row_operations/row_radix_scatter.cpp @@ -173,6 +173,49 @@ void RadixScatterListVector(Vector &v, UnifiedVectorFormat &vdata, const Selecti } } +void RadixScatterArrayVector(Vector &v, UnifiedVectorFormat &vdata, idx_t vcount, const SelectionVector &sel, + idx_t add_count, data_ptr_t *key_locations, const bool desc, const bool has_null, + const bool nulls_first, const idx_t prefix_len, idx_t width, const idx_t offset) { + // serialize null values + if (has_null) { + auto &validity = vdata.validity; + const data_t valid = nulls_first ? 1 : 0; + const data_t invalid = 1 - valid; + + for (idx_t i = 0; i < add_count; i++) { + auto idx = sel.get_index(i); + auto source_idx = vdata.sel->get_index(idx) + offset; + // write validity and according value + if (validity.RowIsValid(source_idx)) { + key_locations[i][0] = valid; + } else { + key_locations[i][0] = invalid; + } + key_locations[i]++; + } + width--; + } + + // serialize the inner child + auto &child_vector = ArrayVector::GetEntry(v); + auto array_size = ArrayType::GetSize(v.GetType()); + for (idx_t i = 0; i < add_count; i++) { + auto idx = sel.get_index(i); + auto source_idx = vdata.sel->get_index(idx) + offset; + auto array_offset = source_idx * array_size; + data_ptr_t key_location = key_locations[i]; + + RowOperations::RadixScatter(child_vector, array_size, *FlatVector::IncrementalSelectionVector(), 1, + key_locations + i, false, true, false, prefix_len, width - 1, array_offset); + // invert bits if desc + if (desc) { + for (idx_t s = 0; s < width; s++) { + *(key_location + s) = ~*(key_location + s); + } + } + } +} + void RadixScatterStructVector(Vector &v, UnifiedVectorFormat &vdata, idx_t vcount, const SelectionVector &sel, idx_t add_count, data_ptr_t *key_locations, const bool desc, const bool has_null, const bool nulls_first, const idx_t prefix_len, idx_t width, const idx_t offset) { @@ -263,6 +306,10 @@ void RowOperations::RadixScatter(Vector &v, idx_t vcount, const SelectionVector RadixScatterStructVector(v, vdata, vcount, sel, ser_count, key_locations, desc, has_null, nulls_first, prefix_len, width, offset); break; + case PhysicalType::ARRAY: + RadixScatterArrayVector(v, vdata, vcount, sel, ser_count, key_locations, desc, has_null, nulls_first, + prefix_len, width, offset); + break; default: throw NotImplementedException("Cannot ORDER BY column with type %s", v.GetType().ToString()); } diff --git a/src/duckdb/src/common/row_operations/row_scatter.cpp b/src/duckdb/src/common/row_operations/row_scatter.cpp index c5104d4f2..14055e47a 100644 --- a/src/duckdb/src/common/row_operations/row_scatter.cpp +++ b/src/duckdb/src/common/row_operations/row_scatter.cpp @@ -146,6 +146,7 @@ void RowOperations::Scatter(DataChunk &columns, UnifiedVectorFormat col_data[], break; case PhysicalType::LIST: case PhysicalType::STRUCT: + case PhysicalType::ARRAY: RowOperations::ComputeEntrySizes(vec, col, entry_sizes, vcount, count, sel); break; default: @@ -217,6 +218,7 @@ void RowOperations::Scatter(DataChunk &columns, UnifiedVectorFormat col_data[], break; case PhysicalType::LIST: case PhysicalType::STRUCT: + case PhysicalType::ARRAY: ScatterNestedVector(vec, col, rows, data_locations, sel, count, col_offset, col_no, vcount); break; default: diff --git a/src/duckdb/src/common/sort/comparators.cpp b/src/duckdb/src/common/sort/comparators.cpp index b6cf7b557..0533fe2d1 100644 --- a/src/duckdb/src/common/sort/comparators.cpp +++ b/src/duckdb/src/common/sort/comparators.cpp @@ -55,6 +55,7 @@ int Comparators::CompareVal(const data_ptr_t l_ptr, const data_ptr_t r_ptr, cons case PhysicalType::VARCHAR: return TemplatedCompareVal(l_ptr, r_ptr); case PhysicalType::LIST: + case PhysicalType::ARRAY: case PhysicalType::STRUCT: { auto l_nested_ptr = Load(l_ptr); auto r_nested_ptr = Load(r_ptr); @@ -146,6 +147,8 @@ int Comparators::CompareValAndAdvance(data_ptr_t &l_ptr, data_ptr_t &r_ptr, cons return CompareListAndAdvance(l_ptr, r_ptr, ListType::GetChildType(type), valid); case PhysicalType::STRUCT: return CompareStructAndAdvance(l_ptr, r_ptr, StructType::GetChildTypes(type), valid); + case PhysicalType::ARRAY: + return CompareArrayAndAdvance(l_ptr, r_ptr, ArrayType::GetChildType(type), valid, ArrayType::GetSize(type)); default: throw NotImplementedException("Unimplemented CompareValAndAdvance for type %s", type.ToString()); } @@ -222,6 +225,119 @@ int Comparators::CompareStructAndAdvance(data_ptr_t &left_ptr, data_ptr_t &right return comp_res; } +int Comparators::CompareArrayAndAdvance(data_ptr_t &left_ptr, data_ptr_t &right_ptr, const LogicalType &type, + bool valid, idx_t array_size) { + if (!valid) { + return 0; + } + + // Load array validity masks + ValidityBytes left_validity(left_ptr); + ValidityBytes right_validity(right_ptr); + left_ptr += (array_size + 7) / 8; + right_ptr += (array_size + 7) / 8; + + int comp_res = 0; + if (TypeIsConstantSize(type.InternalType())) { + // Templated code for fixed-size types + switch (type.InternalType()) { + case PhysicalType::BOOL: + case PhysicalType::INT8: + comp_res = TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::INT16: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::INT32: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::INT64: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::UINT8: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::UINT16: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::UINT32: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::UINT64: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::INT128: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::FLOAT: + comp_res = TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::DOUBLE: + comp_res = TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + case PhysicalType::INTERVAL: + comp_res = + TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, array_size); + break; + default: + throw NotImplementedException("CompareListAndAdvance for fixed-size type %s", type.ToString()); + } + } else { + // Variable-sized array entries + bool left_valid; + bool right_valid; + idx_t entry_idx; + idx_t idx_in_entry; + // Size (in bytes) of all variable-sizes entries is stored before the entries begin, + // to make deserialization easier. We need to skip over them + left_ptr += array_size * sizeof(idx_t); + right_ptr += array_size * sizeof(idx_t); + for (idx_t i = 0; i < array_size; i++) { + ValidityBytes::GetEntryIndex(i, entry_idx, idx_in_entry); + left_valid = left_validity.RowIsValid(left_validity.GetValidityEntry(entry_idx), idx_in_entry); + right_valid = right_validity.RowIsValid(right_validity.GetValidityEntry(entry_idx), idx_in_entry); + if (left_valid && right_valid) { + switch (type.InternalType()) { + case PhysicalType::LIST: + comp_res = CompareListAndAdvance(left_ptr, right_ptr, ListType::GetChildType(type), left_valid); + break; + case PhysicalType::ARRAY: + comp_res = CompareArrayAndAdvance(left_ptr, right_ptr, ArrayType::GetChildType(type), left_valid, + ArrayType::GetSize(type)); + break; + case PhysicalType::VARCHAR: + comp_res = CompareStringAndAdvance(left_ptr, right_ptr, left_valid); + break; + case PhysicalType::STRUCT: + comp_res = + CompareStructAndAdvance(left_ptr, right_ptr, StructType::GetChildTypes(type), left_valid); + break; + default: + throw NotImplementedException("CompareArrayAndAdvance for variable-size type %s", type.ToString()); + } + } else if (!left_valid && !right_valid) { + comp_res = 0; + } else if (left_valid) { + comp_res = -1; + } else { + comp_res = 1; + } + if (comp_res != 0) { + break; + } + } + } + return comp_res; +} + int Comparators::CompareListAndAdvance(data_ptr_t &left_ptr, data_ptr_t &right_ptr, const LogicalType &type, bool valid) { if (!valid) { @@ -302,6 +418,10 @@ int Comparators::CompareListAndAdvance(data_ptr_t &left_ptr, data_ptr_t &right_p case PhysicalType::LIST: comp_res = CompareListAndAdvance(left_ptr, right_ptr, ListType::GetChildType(type), left_valid); break; + case PhysicalType::ARRAY: + comp_res = CompareArrayAndAdvance(left_ptr, right_ptr, ArrayType::GetChildType(type), left_valid, + ArrayType::GetSize(type)); + break; case PhysicalType::VARCHAR: comp_res = CompareStringAndAdvance(left_ptr, right_ptr, left_valid); break; diff --git a/src/duckdb/src/common/sort/sort_state.cpp b/src/duckdb/src/common/sort/sort_state.cpp index b10970d06..b9316c7a7 100644 --- a/src/duckdb/src/common/sort/sort_state.cpp +++ b/src/duckdb/src/common/sort/sort_state.cpp @@ -31,6 +31,10 @@ idx_t GetNestedSortingColSize(idx_t &col_size, const LogicalType &type) { // Structs get 1 bytes (null) col_size++; return GetNestedSortingColSize(col_size, StructType::GetChildType(type, 0)); + case PhysicalType::ARRAY: + // Arrays get 1 bytes (null) + col_size++; + return GetNestedSortingColSize(col_size, ArrayType::GetChildType(type)); default: throw NotImplementedException("Unable to order column with type %s", type.ToString()); } diff --git a/src/duckdb/src/common/string_util.cpp b/src/duckdb/src/common/string_util.cpp index 4f994b6a4..1c2458080 100644 --- a/src/duckdb/src/common/string_util.cpp +++ b/src/duckdb/src/common/string_util.cpp @@ -154,6 +154,22 @@ string StringUtil::Join(const vector &input, const string &separator) { return StringUtil::Join(input, input.size(), separator, [](const string &s) { return s; }); } +string StringUtil::Join(const set &input, const string &separator) { + // The result + std::string result; + + auto it = input.begin(); + while (it != input.end()) { + result += *it; + it++; + if (it == input.end()) { + break; + } + result += separator; + } + return result; +} + string StringUtil::BytesToHumanReadableString(idx_t bytes) { string db_size; auto kilobytes = bytes / 1000; diff --git a/src/duckdb/src/common/types.cpp b/src/duckdb/src/common/types.cpp index 98f6ae2ea..576f5654c 100644 --- a/src/duckdb/src/common/types.cpp +++ b/src/duckdb/src/common/types.cpp @@ -120,6 +120,8 @@ PhysicalType LogicalType::GetInternalType() { case LogicalTypeId::LIST: case LogicalTypeId::MAP: return PhysicalType::LIST; + case LogicalTypeId::ARRAY: + return PhysicalType::ARRAY; case LogicalTypeId::POINTER: // LCOV_EXCL_START if (sizeof(uintptr_t) == sizeof(uint32_t)) { @@ -216,6 +218,11 @@ const vector LogicalType::Integral() { return types; } +const vector LogicalType::Real() { + vector types = {LogicalType::FLOAT, LogicalType::DOUBLE}; + return types; +} + const vector LogicalType::AllTypes() { vector types = { LogicalType::BOOLEAN, LogicalType::TINYINT, LogicalType::SMALLINT, LogicalType::INTEGER, @@ -224,7 +231,7 @@ const vector LogicalType::AllTypes() { LogicalType::INTERVAL, LogicalType::HUGEINT, LogicalTypeId::DECIMAL, LogicalType::UTINYINT, LogicalType::USMALLINT, LogicalType::UINTEGER, LogicalType::UBIGINT, LogicalType::TIME, LogicalTypeId::LIST, LogicalTypeId::STRUCT, LogicalType::TIME_TZ, LogicalType::TIMESTAMP_TZ, - LogicalTypeId::MAP, LogicalTypeId::UNION, LogicalType::UUID}; + LogicalTypeId::MAP, LogicalTypeId::UNION, LogicalType::UUID, LogicalTypeId::ARRAY}; return types; } @@ -265,6 +272,8 @@ string TypeIdToString(PhysicalType type) { return "STRUCT"; case PhysicalType::LIST: return "LIST"; + case PhysicalType::ARRAY: + return "ARRAY"; case PhysicalType::INVALID: return "INVALID"; case PhysicalType::BIT: @@ -309,6 +318,7 @@ idx_t GetTypeIdSize(PhysicalType type) { return sizeof(interval_t); case PhysicalType::STRUCT: case PhysicalType::UNKNOWN: + case PhysicalType::ARRAY: return 0; // no own payload case PhysicalType::LIST: return sizeof(list_entry_t); // offset + len @@ -383,6 +393,17 @@ string LogicalType::ToString() const { ret += ")"; return ret; } + case LogicalTypeId::ARRAY: { + if (!type_info_) { + return "ARRAY"; + } + auto size = ArrayType::GetSize(*this); + if (size == 0) { + return ArrayType::GetChildType(*this).ToString() + "[ANY]"; + } else { + return ArrayType::GetChildType(*this).ToString() + "[" + to_string(size) + "]"; + } + } case LogicalTypeId::DECIMAL: { if (!type_info_) { return "DECIMAL"; @@ -706,6 +727,11 @@ LogicalType LogicalType::MaxLogicalType(const LogicalType &left, const LogicalTy auto new_child = MaxLogicalType(ListType::GetChildType(left), ListType::GetChildType(right)); return LogicalType::LIST(new_child); } + if (type_id == LogicalTypeId::ARRAY) { + auto new_child = MaxLogicalType(ArrayType::GetChildType(left), ArrayType::GetChildType(right)); + auto new_size = MaxValue(ArrayType::GetSize(left), ArrayType::GetSize(right)); + return LogicalType::ARRAY(new_child, new_size); + } if (type_id == LogicalTypeId::MAP) { // list: perform max recursively on child type auto new_child = MaxLogicalType(ListType::GetChildType(left), ListType::GetChildType(right)); @@ -1066,6 +1092,43 @@ PhysicalType EnumType::GetPhysicalType(const LogicalType &type) { return EnumTypeInfo::DictType(info.GetDictSize()); } +//===--------------------------------------------------------------------===// +// Array Type +//===--------------------------------------------------------------------===// + +const LogicalType &ArrayType::GetChildType(const LogicalType &type) { + D_ASSERT(type.id() == LogicalTypeId::ARRAY); + auto info = type.AuxInfo(); + D_ASSERT(info); + return info->Cast().child_type; +} + +idx_t ArrayType::GetSize(const LogicalType &type) { + D_ASSERT(type.id() == LogicalTypeId::ARRAY); + auto info = type.AuxInfo(); + D_ASSERT(info); + return info->Cast().size; +} + +bool ArrayType::IsAnySize(const LogicalType &type) { + D_ASSERT(type.id() == LogicalTypeId::ARRAY); + auto info = type.AuxInfo(); + D_ASSERT(info); + return info->Cast().size == 0; +} + +LogicalType LogicalType::ARRAY(const LogicalType &child, idx_t size) { + D_ASSERT(size > 0); + D_ASSERT(size < ArrayType::MAX_ARRAY_SIZE); + auto info = make_shared(child, size); + return LogicalType(LogicalTypeId::ARRAY, std::move(info)); +} + +LogicalType LogicalType::ARRAY(const LogicalType &child) { + auto info = make_shared(child, 0); + return LogicalType(LogicalTypeId::ARRAY, std::move(info)); +} + //===--------------------------------------------------------------------===// // Logical Type //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/common/types/column/column_data_collection.cpp b/src/duckdb/src/common/types/column/column_data_collection.cpp index 070620fd7..de135432a 100644 --- a/src/duckdb/src/common/types/column/column_data_collection.cpp +++ b/src/duckdb/src/common/types/column/column_data_collection.cpp @@ -636,6 +636,53 @@ void ColumnDataCopyStruct(ColumnDataMetaData &meta_data, const UnifiedVectorForm } } +void ColumnDataCopyArray(ColumnDataMetaData &meta_data, const UnifiedVectorFormat &source_data, Vector &source, + idx_t offset, idx_t copy_count) { + + auto &segment = meta_data.segment; + + // copy the NULL values for the main array vector (the same as for a struct vector) + TemplatedColumnDataCopy(meta_data, source_data, source, offset, copy_count); + + auto &child_vector = ArrayVector::GetEntry(source); + auto &child_type = child_vector.GetType(); + auto array_size = ArrayType::GetSize(source.GetType()); + + if (!meta_data.GetVectorMetaData().child_index.IsValid()) { + auto child_index = segment.AllocateVector(child_type, meta_data.chunk_data, meta_data.state); + meta_data.GetVectorMetaData().child_index = meta_data.segment.AddChildIndex(child_index); + } + + auto &child_function = meta_data.copy_function.child_functions[0]; + auto child_index = segment.GetChildIndex(meta_data.GetVectorMetaData().child_index); + + auto current_child_index = child_index; + while (current_child_index.IsValid()) { + auto &child_vdata = segment.GetVectorData(current_child_index); + current_child_index = child_vdata.next_data; + } + + UnifiedVectorFormat child_vector_data; + ColumnDataMetaData child_meta_data(child_function, meta_data, child_index); + child_vector.ToUnifiedFormat(copy_count * array_size, child_vector_data); + + // Broadcast and sync the validity of the array vector to the child vector + + if (source_data.validity.IsMaskSet()) { + for (idx_t i = 0; i < copy_count; i++) { + auto source_idx = source_data.sel->get_index(offset + i); + if (!source_data.validity.RowIsValid(source_idx)) { + for (idx_t j = 0; j < array_size; j++) { + child_vector_data.validity.SetInvalid(source_idx * array_size + j); + } + } + } + } + + child_function.function(child_meta_data, child_vector_data, child_vector, offset * array_size, + array_size * copy_count); +} + ColumnDataCopyFunction ColumnDataCollection::GetCopyFunction(const LogicalType &type) { ColumnDataCopyFunction result; column_data_copy_function_t function; @@ -696,6 +743,12 @@ ColumnDataCopyFunction ColumnDataCollection::GetCopyFunction(const LogicalType & result.child_functions.push_back(child_function); break; } + case PhysicalType::ARRAY: { + function = ColumnDataCopyArray; + auto child_function = GetCopyFunction(ArrayType::GetChildType(type)); + result.child_functions.push_back(child_function); + break; + } default: throw InternalException("Unsupported type for ColumnDataCollection::GetCopyFunction"); } @@ -707,6 +760,7 @@ static bool IsComplexType(const LogicalType &type) { switch (type.InternalType()) { case PhysicalType::STRUCT: case PhysicalType::LIST: + case PhysicalType::ARRAY: return true; default: return false; diff --git a/src/duckdb/src/common/types/column/column_data_collection_segment.cpp b/src/duckdb/src/common/types/column/column_data_collection_segment.cpp index 2f534dc9c..f1f3d4af1 100644 --- a/src/duckdb/src/common/types/column/column_data_collection_segment.cpp +++ b/src/duckdb/src/common/types/column/column_data_collection_segment.cpp @@ -24,7 +24,9 @@ VectorDataIndex ColumnDataCollectionSegment::AllocateVectorInternal(const Logica meta_data.count = 0; auto internal_type = type.InternalType(); - auto type_size = internal_type == PhysicalType::STRUCT ? 0 : GetTypeIdSize(internal_type); + auto type_size = ((internal_type == PhysicalType::STRUCT) || (internal_type == PhysicalType::ARRAY)) + ? 0 + : GetTypeIdSize(internal_type); allocator->AllocateData(GetDataSize(type_size) + ValidityMask::STANDARD_MASK_SIZE, meta_data.block_id, meta_data.offset, chunk_state); if (allocator->GetType() == ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR || @@ -194,6 +196,11 @@ idx_t ColumnDataCollectionSegment::ReadVector(ChunkManagementState &state, Vecto auto &child_vector = ListVector::GetEntry(result); auto child_count = ReadVector(state, GetChildIndex(vdata.child_index), child_vector); ListVector::SetListSize(result, child_count); + + } else if (internal_type == PhysicalType::ARRAY) { + auto &child_vector = ArrayVector::GetEntry(result); + auto child_count = ReadVector(state, GetChildIndex(vdata.child_index), child_vector); + (void)child_count; } else if (internal_type == PhysicalType::STRUCT) { auto &child_vectors = StructVector::GetEntries(result); for (idx_t child_idx = 0; child_idx < child_vectors.size(); child_idx++) { diff --git a/src/duckdb/src/common/types/list_segment.cpp b/src/duckdb/src/common/types/list_segment.cpp index 2a14718bf..6627383af 100644 --- a/src/duckdb/src/common/types/list_segment.cpp +++ b/src/duckdb/src/common/types/list_segment.cpp @@ -58,6 +58,28 @@ static LinkedList *GetListChildData(ListSegment *segment) { segment->capacity * (sizeof(bool) + sizeof(uint64_t))); } +//===--------------------------------------------------------------------===// +// Array +//===--------------------------------------------------------------------===// +static idx_t GetAllocationSizeArray(uint16_t capacity) { + // Only store the null mask for the array segment, length is fixed so we don't need to store it + return AlignValue(sizeof(ListSegment) + capacity * (sizeof(bool)) + sizeof(LinkedList)); +} + +static data_ptr_t AllocateArrayData(ArenaAllocator &allocator, uint16_t capacity) { + return allocator.Allocate(GetAllocationSizeArray(capacity)); +} + +static const LinkedList *GetArrayChildData(const ListSegment *segment) { + return reinterpret_cast(const_data_ptr_cast(segment) + sizeof(ListSegment) + + segment->capacity * sizeof(bool)); +} + +static LinkedList *GetArrayChildData(ListSegment *segment) { + return reinterpret_cast(data_ptr_cast(segment) + sizeof(ListSegment) + + segment->capacity * sizeof(bool)); +} + //===--------------------------------------------------------------------===// // Structs //===--------------------------------------------------------------------===// @@ -143,6 +165,22 @@ static ListSegment *CreateStructSegment(const ListSegmentFunctions &functions, A return segment; } +static ListSegment *CreateArraySegment(const ListSegmentFunctions &, ArenaAllocator &allocator, uint16_t capacity) { + // allocate data and set header + auto segment = reinterpret_cast(AllocateArrayData(allocator, capacity)); + + segment->capacity = capacity; + segment->count = 0; + segment->next = nullptr; + + // create an empty linked list for the child vector + auto linked_child_list = GetArrayChildData(segment); + LinkedList linked_list(0, nullptr, nullptr); + Store(linked_list, data_ptr_cast(linked_child_list)); + + return segment; +} + static ListSegment *GetSegment(const ListSegmentFunctions &functions, ArenaAllocator &allocator, LinkedList &linked_list) { ListSegment *segment; @@ -291,6 +329,31 @@ static void WriteDataToStructSegment(const ListSegmentFunctions &functions, Aren } } +static void WriteDataToArraySegment(const ListSegmentFunctions &functions, ArenaAllocator &allocator, + ListSegment *segment, RecursiveUnifiedVectorFormat &input_data, idx_t &entry_idx) { + auto sel_entry_idx = input_data.unified.sel->get_index(entry_idx); + + // write null validity + auto null_mask = GetNullMask(segment); + auto valid = input_data.unified.validity.RowIsValid(sel_entry_idx); + null_mask[segment->count] = !valid; + + if (!valid) { + return; + } + + auto array_size = ArrayType::GetSize(input_data.logical_type); + auto array_offset = sel_entry_idx * array_size; + + auto child_segments = Load(data_ptr_cast(GetArrayChildData(segment))); + D_ASSERT(functions.child_functions.size() == 1); + for (idx_t elem_idx = array_offset; elem_idx < array_offset + array_size; elem_idx++) { + functions.child_functions[0].AppendRow(allocator, child_segments, input_data.children.back(), elem_idx); + } + // store the updated linked list + Store(child_segments, data_ptr_cast(GetArrayChildData(segment))); +} + void ListSegmentFunctions::AppendRow(ArenaAllocator &allocator, LinkedList &linked_list, RecursiveUnifiedVectorFormat &input_data, idx_t &entry_idx) const { @@ -437,6 +500,29 @@ static void ReadDataFromStructSegment(const ListSegmentFunctions &functions, con } } +static void ReadDataFromArraySegment(const ListSegmentFunctions &functions, const ListSegment *segment, Vector &result, + idx_t &total_count) { + + auto &aggr_vector_validity = FlatVector::Validity(result); + + // set NULLs + auto null_mask = GetNullMask(segment); + for (idx_t i = 0; i < segment->count; i++) { + if (null_mask[i]) { + aggr_vector_validity.SetInvalid(total_count + i); + } + } + + auto &child_vector = ArrayVector::GetEntry(result); + auto linked_child_list = Load(const_data_ptr_cast(GetArrayChildData(segment))); + auto array_size = ArrayType::GetSize(result.GetType()); + auto child_size = array_size * total_count; + + // recurse into the linked list of child values + D_ASSERT(functions.child_functions.size() == 1); + functions.child_functions[0].BuildListVector(linked_child_list, child_vector, child_size); +} + void ListSegmentFunctions::BuildListVector(const LinkedList &linked_list, Vector &result, idx_t &initial_total_count) const { auto &read_data_from_segment = *this; @@ -444,7 +530,6 @@ void ListSegmentFunctions::BuildListVector(const LinkedList &linked_list, Vector auto segment = linked_list.first_segment; while (segment) { read_data_from_segment.read_data(read_data_from_segment, segment, result, total_count); - total_count += segment->count; segment = segment->next; } @@ -540,6 +625,16 @@ void GetSegmentDataFunctions(ListSegmentFunctions &functions, const LogicalType } break; } + case PhysicalType::ARRAY: { + functions.create_segment = CreateArraySegment; + functions.write_data = WriteDataToArraySegment; + functions.read_data = ReadDataFromArraySegment; + + // recurse + functions.child_functions.emplace_back(); + GetSegmentDataFunctions(functions.child_functions.back(), ArrayType::GetChildType(type)); + break; + } default: throw InternalException("LIST aggregate not yet implemented for " + type.ToString()); } diff --git a/src/duckdb/src/common/types/row/tuple_data_allocator.cpp b/src/duckdb/src/common/types/row/tuple_data_allocator.cpp index b30af49cb..ce5c4b522 100644 --- a/src/duckdb/src/common/types/row/tuple_data_allocator.cpp +++ b/src/duckdb/src/common/types/row/tuple_data_allocator.cpp @@ -347,7 +347,8 @@ void TupleDataAllocator::RecomputeHeapPointers(Vector &old_heap_ptrs, const Sele VerifyStrings(type.id(), row_locations, col_idx, base_col_offset, col_offset, offset, count); break; } - case PhysicalType::LIST: { + case PhysicalType::LIST: + case PhysicalType::ARRAY: { for (idx_t i = 0; i < count; i++) { const auto idx = offset + i; const auto &row_location = row_locations[idx] + base_col_offset; diff --git a/src/duckdb/src/common/types/row/tuple_data_collection.cpp b/src/duckdb/src/common/types/row/tuple_data_collection.cpp index 97858151a..f8ab6df14 100644 --- a/src/duckdb/src/common/types/row/tuple_data_collection.cpp +++ b/src/duckdb/src/common/types/row/tuple_data_collection.cpp @@ -87,7 +87,8 @@ void VerifyAppendColumns(const TupleDataLayout &layout, const vector & } // This column will not be appended in the first go - verify that it is fixed-size - we cannot resize heap after const auto physical_type = layout.GetTypes()[col_idx].InternalType(); - D_ASSERT(physical_type != PhysicalType::VARCHAR && physical_type != PhysicalType::LIST); + D_ASSERT(physical_type != PhysicalType::VARCHAR && physical_type != PhysicalType::LIST && + physical_type != PhysicalType::ARRAY); if (physical_type == PhysicalType::STRUCT) { const auto &struct_layout = layout.GetStructLayout(col_idx); vector struct_column_ids; @@ -140,6 +141,9 @@ static void InitializeVectorFormat(vector &vector_data, c case PhysicalType::LIST: InitializeVectorFormat(vector_data[col_idx].children, {ListType::GetChildType(type)}); break; + case PhysicalType::ARRAY: + InitializeVectorFormat(vector_data[col_idx].children, {ArrayType::GetChildType(type)}); + break; default: break; } @@ -238,6 +242,29 @@ static inline void ToUnifiedFormatInternal(TupleDataVectorFormat &format, Vector ToUnifiedFormatInternal(reinterpret_cast(format.children[0]), ListVector::GetEntry(vector), ListVector::GetListSize(vector)); break; + case PhysicalType::ARRAY: { + D_ASSERT(format.children.size() == 1); + + // For arrays, we cheat a bit and pretend that they are lists by creating and assigning list_entry_t's to the + // vector This allows us to reuse all the list serialization functions for array types too. + + // This is kind of hacky, but we need to create a list_entry_t for each array entry + idx_t array_count = ArrayVector::GetTotalSize(vector) / ArrayType::GetSize(vector.GetType()); + format.array_list_entries = make_uniq_array(array_count); + + auto array_size = ArrayType::GetSize(vector.GetType()); + // create list entries + for (idx_t i = 0; i < array_count; i++) { + format.array_list_entries[i].length = array_size; + format.array_list_entries[i].offset = i * array_size; + } + format.unified.data = reinterpret_cast(format.array_list_entries.get()); + + // Set the array size in the child format + format.children[0].parent_array_size = array_size; + + ToUnifiedFormatInternal(format.children[0], ArrayVector::GetEntry(vector), ArrayVector::GetTotalSize(vector)); + } break; default: break; } diff --git a/src/duckdb/src/common/types/row/tuple_data_scatter_gather.cpp b/src/duckdb/src/common/types/row/tuple_data_scatter_gather.cpp index 849d5d480..aa4ceec03 100644 --- a/src/duckdb/src/common/types/row/tuple_data_scatter_gather.cpp +++ b/src/duckdb/src/common/types/row/tuple_data_scatter_gather.cpp @@ -96,7 +96,8 @@ void TupleDataCollection::ComputeHeapSizes(Vector &heap_sizes_v, const Vector &s TupleDataVectorFormat &source_format, const SelectionVector &append_sel, const idx_t append_count) { const auto type = source_v.GetType().InternalType(); - if (type != PhysicalType::VARCHAR && type != PhysicalType::STRUCT && type != PhysicalType::LIST) { + if (type != PhysicalType::VARCHAR && type != PhysicalType::STRUCT && type != PhysicalType::LIST && + type != PhysicalType::ARRAY) { return; } @@ -144,8 +145,25 @@ void TupleDataCollection::ComputeHeapSizes(Vector &heap_sizes_v, const Vector &s D_ASSERT(source_format.children.size() == 1); auto &child_source_v = ListVector::GetEntry(source_v); auto &child_format = source_format.children[0]; - TupleDataCollection::WithinListHeapComputeSizes(heap_sizes_v, child_source_v, child_format, append_sel, - append_count, source_vector_data); + TupleDataCollection::WithinCollectionComputeHeapSizes(heap_sizes_v, child_source_v, child_format, append_sel, + append_count, source_vector_data); + break; + } + case PhysicalType::ARRAY: { + // Arrays are stored entirely in the heap + for (idx_t i = 0; i < append_count; i++) { + auto source_idx = source_sel.get_index(append_sel.get_index(i)); + if (source_validity.RowIsValid(source_idx)) { + heap_sizes[i] += sizeof(uint64_t); // Size of the list + } + } + + // Recurse + D_ASSERT(source_format.children.size() == 1); + auto &child_source_v = ArrayVector::GetEntry(source_v); + auto &child_format = source_format.children[0]; + TupleDataCollection::WithinCollectionComputeHeapSizes(heap_sizes_v, child_source_v, child_format, append_sel, + append_count, source_vector_data); break; } default: @@ -153,39 +171,43 @@ void TupleDataCollection::ComputeHeapSizes(Vector &heap_sizes_v, const Vector &s } } -void TupleDataCollection::WithinListHeapComputeSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, - const SelectionVector &append_sel, const idx_t append_count, - const UnifiedVectorFormat &list_data) { +void TupleDataCollection::WithinCollectionComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, const idx_t append_count, + const UnifiedVectorFormat &list_data) { auto type = source_v.GetType().InternalType(); if (TypeIsConstantSize(type)) { - TupleDataCollection::ComputeFixedWithinListHeapSizes(heap_sizes_v, source_v, source_format, append_sel, - append_count, list_data); + TupleDataCollection::ComputeFixedWithinCollectionHeapSizes(heap_sizes_v, source_v, source_format, append_sel, + append_count, list_data); return; } - switch (type) { case PhysicalType::VARCHAR: - TupleDataCollection::StringWithinListComputeHeapSizes(heap_sizes_v, source_v, source_format, append_sel, - append_count, list_data); + TupleDataCollection::StringWithinCollectionComputeHeapSizes(heap_sizes_v, source_v, source_format, append_sel, + append_count, list_data); break; case PhysicalType::STRUCT: - TupleDataCollection::StructWithinListComputeHeapSizes(heap_sizes_v, source_v, source_format, append_sel, - append_count, list_data); + TupleDataCollection::StructWithinCollectionComputeHeapSizes(heap_sizes_v, source_v, source_format, append_sel, + append_count, list_data); break; case PhysicalType::LIST: - TupleDataCollection::ListWithinListComputeHeapSizes(heap_sizes_v, source_v, source_format, append_sel, - append_count, list_data); + TupleDataCollection::CollectionWithinCollectionComputeHeapSizes(heap_sizes_v, source_v, source_format, + append_sel, append_count, list_data); + break; + case PhysicalType::ARRAY: + TupleDataCollection::CollectionWithinCollectionComputeHeapSizes(heap_sizes_v, source_v, source_format, + append_sel, append_count, list_data); break; default: throw NotImplementedException("WithinListHeapComputeSizes for %s", EnumUtil::ToString(source_v.GetType().id())); } } -void TupleDataCollection::ComputeFixedWithinListHeapSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, - const SelectionVector &append_sel, const idx_t append_count, - const UnifiedVectorFormat &list_data) { +void TupleDataCollection::ComputeFixedWithinCollectionHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, + const idx_t append_count, + const UnifiedVectorFormat &list_data) { // List data const auto list_sel = *list_data.sel; const auto list_entries = UnifiedVectorFormat::GetData(list_data); @@ -212,10 +234,11 @@ void TupleDataCollection::ComputeFixedWithinListHeapSizes(Vector &heap_sizes_v, } } -void TupleDataCollection::StringWithinListComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, - const SelectionVector &append_sel, const idx_t append_count, - const UnifiedVectorFormat &list_data) { +void TupleDataCollection::StringWithinCollectionComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, + const idx_t append_count, + const UnifiedVectorFormat &list_data) { // Source const auto &source_data = source_format.unified; const auto &source_sel = *source_data.sel; @@ -256,10 +279,11 @@ void TupleDataCollection::StringWithinListComputeHeapSizes(Vector &heap_sizes_v, } } -void TupleDataCollection::StructWithinListComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, - const SelectionVector &append_sel, const idx_t append_count, - const UnifiedVectorFormat &list_data) { +void TupleDataCollection::StructWithinCollectionComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, + const idx_t append_count, + const UnifiedVectorFormat &list_data) { // List data const auto list_sel = *list_data.sel; const auto list_entries = UnifiedVectorFormat::GetData(list_data); @@ -285,9 +309,10 @@ void TupleDataCollection::StructWithinListComputeHeapSizes(Vector &heap_sizes_v, auto &struct_sources = StructVector::GetEntries(source_v); for (idx_t struct_col_idx = 0; struct_col_idx < struct_sources.size(); struct_col_idx++) { auto &struct_source = *struct_sources[struct_col_idx]; + auto &struct_format = source_format.children[struct_col_idx]; - TupleDataCollection::WithinListHeapComputeSizes(heap_sizes_v, struct_source, struct_format, append_sel, - append_count, list_data); + TupleDataCollection::WithinCollectionComputeHeapSizes(heap_sizes_v, struct_source, struct_format, append_sel, + append_count, list_data); } } @@ -317,10 +342,11 @@ static void ApplySliceRecursive(const Vector &source_v, TupleDataVectorFormat &s } } -void TupleDataCollection::ListWithinListComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, - const SelectionVector &append_sel, const idx_t append_count, - const UnifiedVectorFormat &list_data) { +void TupleDataCollection::CollectionWithinCollectionComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, + const idx_t append_count, + const UnifiedVectorFormat &list_data) { // List data (of the list Vector that "source_v" is in) const auto list_sel = *list_data.sel; const auto list_entries = UnifiedVectorFormat::GetData(list_data); @@ -356,7 +382,10 @@ void TupleDataCollection::ListWithinListComputeHeapSizes(Vector &heap_sizes_v, c sum_of_sizes += child_list_length; } } - const auto child_list_child_count = MaxValue(sum_of_sizes, ListVector::GetListSize(source_v)); + + const auto child_list_child_count = MaxValue( + sum_of_sizes, source_v.GetType().InternalType() == PhysicalType::LIST ? ListVector::GetListSize(source_v) + : ArrayVector::GetTotalSize(source_v)); // Target auto heap_sizes = FlatVector::GetData(heap_sizes_v); @@ -426,12 +455,16 @@ void TupleDataCollection::ListWithinListComputeHeapSizes(Vector &heap_sizes_v, c // Combine the selection vectors D_ASSERT(source_format.children.size() == 1); - auto &child_source = ListVector::GetEntry(source_v); + + // TODO: Template this? + auto &child_source = source_v.GetType().InternalType() == PhysicalType::LIST ? ListVector::GetEntry(source_v) + : ArrayVector::GetEntry(source_v); + ApplySliceRecursive(child_source, child_format, combined_sel, child_list_child_count); // Recurse - TupleDataCollection::WithinListHeapComputeSizes(heap_sizes_v, child_source, child_format, append_sel, append_count, - combined_child_list_data); + TupleDataCollection::WithinCollectionComputeHeapSizes(heap_sizes_v, child_source, child_format, append_sel, + append_count, combined_child_list_data); } void TupleDataCollection::Scatter(TupleDataChunkState &chunk_state, const DataChunk &new_chunk, @@ -564,6 +597,10 @@ static void TupleDataStructScatter(const Vector &source, const TupleDataVectorFo } } +//------------------------------------------------------------------------------ +// List Scatter +//------------------------------------------------------------------------------ + static void TupleDataListScatter(const Vector &source, const TupleDataVectorFormat &source_format, const SelectionVector &append_sel, const idx_t append_count, const TupleDataLayout &layout, const Vector &row_locations, Vector &heap_locations, @@ -609,13 +646,66 @@ static void TupleDataListScatter(const Vector &source, const TupleDataVectorForm col_idx, source_format.unified, child_function.child_functions); } +//------------------------------------------------------------------------------ +// Array Scatter +//------------------------------------------------------------------------------ + +static void TupleDataArrayScatter(const Vector &source, const TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, const idx_t append_count, + const TupleDataLayout &layout, const Vector &row_locations, Vector &heap_locations, + const idx_t col_idx, const UnifiedVectorFormat &dummy_arg, + const vector &child_functions) { + // Source + // The Array vector has fake list_entry_t's set by this point, so this is fine + const auto &source_data = source_format.unified; + const auto source_sel = *source_data.sel; + const auto data = UnifiedVectorFormat::GetData(source_data); + const auto &validity = source_data.validity; + + // Target + auto target_locations = FlatVector::GetData(row_locations); + auto target_heap_locations = FlatVector::GetData(heap_locations); + + // Precompute mask indexes + idx_t entry_idx; + idx_t idx_in_entry; + ValidityBytes::GetEntryIndex(col_idx, entry_idx, idx_in_entry); + + // Set validity of the LIST in this layout, and store pointer to where it's stored + const auto offset_in_row = layout.GetOffsets()[col_idx]; + for (idx_t i = 0; i < append_count; i++) { + const auto source_idx = source_sel.get_index(append_sel.get_index(i)); + if (validity.RowIsValid(source_idx)) { + auto &target_heap_location = target_heap_locations[i]; + Store(target_heap_location, target_locations[i] + offset_in_row); + + // Store list length and skip over it + Store(data[source_idx].length, target_heap_location); + target_heap_location += sizeof(uint64_t); + } else { + ValidityBytes(target_locations[i]).SetInvalidUnsafe(entry_idx, idx_in_entry); + } + } + + // Recurse + D_ASSERT(child_functions.size() == 1); + auto &child_source = ArrayVector::GetEntry(source); + auto &child_format = source_format.children[0]; + const auto &child_function = child_functions[0]; + child_function.function(child_source, child_format, append_sel, append_count, layout, row_locations, heap_locations, + col_idx, source_format.unified, child_function.child_functions); +} + +//------------------------------------------------------------------------------ +// Collection Scatter +//------------------------------------------------------------------------------ template -static void TupleDataTemplatedWithinListScatter(const Vector &source, const TupleDataVectorFormat &source_format, - const SelectionVector &append_sel, const idx_t append_count, - const TupleDataLayout &layout, const Vector &row_locations, - Vector &heap_locations, const idx_t col_idx, - const UnifiedVectorFormat &list_data, - const vector &child_functions) { +static void TupleDataTemplatedWithinCollectionScatter(const Vector &source, const TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, const idx_t append_count, + const TupleDataLayout &layout, const Vector &row_locations, + Vector &heap_locations, const idx_t col_idx, + const UnifiedVectorFormat &list_data, + const vector &child_functions) { // Source const auto &source_data = source_format.unified; const auto &source_sel = *source_data.sel; @@ -665,12 +755,12 @@ static void TupleDataTemplatedWithinListScatter(const Vector &source, const Tupl } } -static void TupleDataStructWithinListScatter(const Vector &source, const TupleDataVectorFormat &source_format, - const SelectionVector &append_sel, const idx_t append_count, - const TupleDataLayout &layout, const Vector &row_locations, - Vector &heap_locations, const idx_t col_idx, - const UnifiedVectorFormat &list_data, - const vector &child_functions) { +static void TupleDataStructWithinCollectionScatter(const Vector &source, const TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, const idx_t append_count, + const TupleDataLayout &layout, const Vector &row_locations, + Vector &heap_locations, const idx_t col_idx, + const UnifiedVectorFormat &list_data, + const vector &child_functions) { // Source const auto &source_data = source_format.unified; const auto &source_sel = *source_data.sel; @@ -723,12 +813,14 @@ static void TupleDataStructWithinListScatter(const Vector &source, const TupleDa } } -static void TupleDataListWithinListScatter(const Vector &child_list, const TupleDataVectorFormat &child_list_format, - const SelectionVector &append_sel, const idx_t append_count, - const TupleDataLayout &layout, const Vector &row_locations, - Vector &heap_locations, const idx_t col_idx, - const UnifiedVectorFormat &list_data, - const vector &child_functions) { +template +static void TupleDataCollectionWithinCollectionScatter(const Vector &child_list, + const TupleDataVectorFormat &child_list_format, + const SelectionVector &append_sel, const idx_t append_count, + const TupleDataLayout &layout, const Vector &row_locations, + Vector &heap_locations, const idx_t col_idx, + const UnifiedVectorFormat &list_data, + const vector &child_functions) { // List data (of the list Vector that "child_list" is in) const auto list_sel = *list_data.sel; const auto list_entries = UnifiedVectorFormat::GetData(list_data); @@ -777,7 +869,7 @@ static void TupleDataListWithinListScatter(const Vector &child_list, const Tuple // Recurse D_ASSERT(child_functions.size() == 1); - auto &child_vec = ListVector::GetEntry(child_list); + auto &child_vec = COLLECTION::GetEntry(child_list); auto &child_format = child_list_format.children[0]; auto &combined_child_list_data = child_format.combined_list_data->combined_data; const auto &child_function = child_functions[0]; @@ -785,66 +877,116 @@ static void TupleDataListWithinListScatter(const Vector &child_list, const Tuple col_idx, combined_child_list_data, child_function.child_functions); } +//------------------------------------------------------------------------------ +// Get Scatter Function +//------------------------------------------------------------------------------ template -tuple_data_scatter_function_t TupleDataGetScatterFunction(bool within_list) { - return within_list ? TupleDataTemplatedWithinListScatter : TupleDataTemplatedScatter; +tuple_data_scatter_function_t TupleDataGetScatterFunction(WithinCollection within_collection) { + switch (within_collection) { + case WithinCollection::NO: + return TupleDataTemplatedScatter; + case WithinCollection::ARRAY: + case WithinCollection::LIST: + return TupleDataTemplatedWithinCollectionScatter; + default: + throw NotImplementedException("Unimplemented within collection type"); + } } -TupleDataScatterFunction TupleDataCollection::GetScatterFunction(const LogicalType &type, bool within_list) { +TupleDataScatterFunction TupleDataCollection::GetScatterFunction(const LogicalType &type, + WithinCollection within_collection) { TupleDataScatterFunction result; switch (type.InternalType()) { case PhysicalType::BOOL: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::INT8: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::INT16: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::INT32: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::INT64: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::INT128: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::UINT8: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::UINT16: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::UINT32: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::UINT64: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::FLOAT: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::DOUBLE: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::INTERVAL: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::VARCHAR: - result.function = TupleDataGetScatterFunction(within_list); + result.function = TupleDataGetScatterFunction(within_collection); break; case PhysicalType::STRUCT: { - result.function = within_list ? TupleDataStructWithinListScatter : TupleDataStructScatter; + switch (within_collection) { + case WithinCollection::NO: + result.function = TupleDataStructScatter; + break; + case WithinCollection::LIST: + case WithinCollection::ARRAY: + result.function = TupleDataStructWithinCollectionScatter; + break; + default: + throw NotImplementedException("Unimplemented within collection type"); + } for (const auto &child_type : StructType::GetChildTypes(type)) { - result.child_functions.push_back(GetScatterFunction(child_type.second, within_list)); + result.child_functions.push_back(GetScatterFunction(child_type.second, within_collection)); } break; } case PhysicalType::LIST: - result.function = within_list ? TupleDataListWithinListScatter : TupleDataListScatter; - result.child_functions.emplace_back(GetScatterFunction(ListType::GetChildType(type), true)); + switch (within_collection) { + case WithinCollection::NO: + result.function = TupleDataListScatter; + break; + case WithinCollection::LIST: + result.function = TupleDataCollectionWithinCollectionScatter; + break; + case WithinCollection::ARRAY: + result.function = TupleDataCollectionWithinCollectionScatter; + break; + default: + throw NotImplementedException("Unimplemented within collection type"); + } + result.child_functions.emplace_back(GetScatterFunction(ListType::GetChildType(type), WithinCollection::LIST)); + break; + case PhysicalType::ARRAY: + switch (within_collection) { + case WithinCollection::NO: + result.function = TupleDataArrayScatter; + break; + case WithinCollection::LIST: + result.function = TupleDataCollectionWithinCollectionScatter; + break; + case WithinCollection::ARRAY: + result.function = TupleDataCollectionWithinCollectionScatter; + break; + default: + throw NotImplementedException("Unimplemented within collection type"); + } + result.child_functions.emplace_back(GetScatterFunction(ArrayType::GetChildType(type), WithinCollection::ARRAY)); break; default: throw InternalException("Unsupported type for TupleDataCollection::GetScatterFunction"); @@ -852,6 +994,10 @@ TupleDataScatterFunction TupleDataCollection::GetScatterFunction(const LogicalTy return result; } +//------------------------------------------------------------------------------- +// Gather +//------------------------------------------------------------------------------- + void TupleDataCollection::Gather(Vector &row_locations, const SelectionVector &scan_sel, const idx_t scan_count, DataChunk &result, const SelectionVector &target_sel) const { D_ASSERT(result.ColumnCount() == layout.ColumnCount()); @@ -957,6 +1103,9 @@ static void TupleDataStructGather(const TupleDataLayout &layout, Vector &row_loc } } +//------------------------------------------------------------------------------ +// List Gather +//------------------------------------------------------------------------------ static void TupleDataListGather(const TupleDataLayout &layout, Vector &row_locations, const idx_t col_idx, const SelectionVector &scan_sel, const idx_t scan_count, Vector &target, const SelectionVector &target_sel, Vector &dummy_vector, @@ -1013,12 +1162,80 @@ static void TupleDataListGather(const TupleDataLayout &layout, Vector &row_locat ListVector::GetEntry(target), target_sel, target, child_function.child_functions); } +static void TupleDataArrayGather(const TupleDataLayout &layout, Vector &row_locations, const idx_t col_idx, + const SelectionVector &scan_sel, const idx_t scan_count, Vector &target, + const SelectionVector &target_sel, Vector &dummy_vector, + const vector &child_functions) { + // Source + auto source_locations = FlatVector::GetData(row_locations); + + // Setup fake list_entry_t's for the target + ArrayVector::AllocateDummyListEntries(target); + + // Target + auto target_list_entries = FlatVector::GetData(target); + auto &target_validity = FlatVector::Validity(target); + auto &target_child = ArrayVector::GetEntry(target); + auto &target_child_validity = FlatVector::Validity(target_child); + auto target_array_size = ArrayType::GetSize(target.GetType()); + + // Precompute mask indexes + idx_t entry_idx; + idx_t idx_in_entry; + ValidityBytes::GetEntryIndex(col_idx, entry_idx, idx_in_entry); + + // Load pointers to the data from the row + Vector heap_locations(LogicalType::POINTER); + auto source_heap_locations = FlatVector::GetData(heap_locations); + auto &source_heap_validity = FlatVector::Validity(heap_locations); + + const auto offset_in_row = layout.GetOffsets()[col_idx]; + uint64_t target_list_offset = 0; + for (idx_t i = 0; i < scan_count; i++) { + const auto source_idx = scan_sel.get_index(i); + const auto target_idx = target_sel.get_index(i); + + const auto &source_row = source_locations[source_idx]; + ValidityBytes row_mask(source_row); + if (row_mask.RowIsValid(row_mask.GetValidityEntry(entry_idx), idx_in_entry)) { + auto &source_heap_location = source_heap_locations[source_idx]; + source_heap_location = Load(source_row + offset_in_row); + + // Load list size and skip over + const auto list_length = Load(source_heap_location); + source_heap_location += sizeof(uint64_t); + + // Initialize list entry, and increment offset + target_list_entries[target_idx] = {target_list_offset, list_length}; + target_list_offset += list_length; + } else { + source_heap_validity.SetInvalid(source_idx); + target_validity.SetInvalid(target_idx); + // We also need to invalidate the corresponding elements in the child array. + for (idx_t elem_idx = 0; elem_idx < target_array_size; elem_idx++) { + target_child_validity.SetInvalid(target_idx * target_array_size + elem_idx); + } + } + } + + auto list_size_before = 0; + + // Recurse + D_ASSERT(child_functions.size() == 1); + const auto &child_function = child_functions[0]; + child_function.function(layout, heap_locations, list_size_before, scan_sel, scan_count, target_child, target_sel, + target, child_function.child_functions); +} + +//------------------------------------------------------------------------------ +// Collection Gather +//------------------------------------------------------------------------------ template -static void TupleDataTemplatedWithinListGather(const TupleDataLayout &layout, Vector &heap_locations, - const idx_t list_size_before, const SelectionVector &scan_sel, - const idx_t scan_count, Vector &target, - const SelectionVector &target_sel, Vector &list_vector, - const vector &child_functions) { +static void TupleDataTemplatedWithinCollectionGather(const TupleDataLayout &layout, Vector &heap_locations, + const idx_t list_size_before, const SelectionVector &scan_sel, + const idx_t scan_count, Vector &target, + const SelectionVector &target_sel, Vector &list_vector, + const vector &child_functions) { // Source auto source_heap_locations = FlatVector::GetData(heap_locations); auto &source_heap_validity = FlatVector::Validity(heap_locations); @@ -1061,11 +1278,11 @@ static void TupleDataTemplatedWithinListGather(const TupleDataLayout &layout, Ve } } -static void TupleDataStructWithinListGather(const TupleDataLayout &layout, Vector &heap_locations, - const idx_t list_size_before, const SelectionVector &scan_sel, - const idx_t scan_count, Vector &target, const SelectionVector &target_sel, - Vector &list_vector, - const vector &child_functions) { +static void TupleDataStructWithinCollectionGather(const TupleDataLayout &layout, Vector &heap_locations, + const idx_t list_size_before, const SelectionVector &scan_sel, + const idx_t scan_count, Vector &target, + const SelectionVector &target_sel, Vector &list_vector, + const vector &child_functions) { // Source auto source_heap_locations = FlatVector::GetData(heap_locations); auto &source_heap_validity = FlatVector::Validity(heap_locations); @@ -1109,18 +1326,75 @@ static void TupleDataStructWithinListGather(const TupleDataLayout &layout, Vecto } } -static void TupleDataListWithinListGather(const TupleDataLayout &layout, Vector &heap_locations, - const idx_t list_size_before, const SelectionVector &scan_sel, - const idx_t scan_count, Vector &target, const SelectionVector &target_sel, - Vector &list_vector, const vector &child_functions) { +template +struct CollectionVector { + static inline void Setup(Vector &collection) = delete; + static inline idx_t GetSize(Vector &collection) = delete; + static inline idx_t GetSizeBefore(Vector &collection) = delete; + static inline Vector &GetEntry(Vector &collection) = delete; + static inline void Reserve(Vector &collection, const idx_t new_capacity) = delete; + static inline void SetSize(Vector &collection, const idx_t new_size) = delete; +}; + +template <> +struct CollectionVector { + static inline void Setup(Vector &collection) { + ArrayVector::AllocateDummyListEntries(collection); + } + static inline idx_t GetSize(Vector &collection) { + return ArrayVector::GetTotalSize(collection); + } + static inline idx_t GetSizeBefore(Vector &) { + return 0; + } + static inline Vector &GetEntry(Vector &collection) { + return ArrayVector::GetEntry(collection); + } + static inline void Reserve(Vector &, const idx_t) { + } + static inline void SetSize(Vector &, const idx_t) { + } +}; + +template <> +struct CollectionVector { + static inline void Setup(Vector &collection) { + // do nothing + } + static inline idx_t GetSize(Vector &collection) { + return ListVector::GetListSize(collection); + } + static inline idx_t GetSizeBefore(Vector &collection) { + return ListVector::GetListSize(collection); + } + static inline Vector &GetEntry(Vector &collection) { + return ListVector::GetEntry(collection); + } + static inline void Reserve(Vector &collection, const idx_t new_capacity) { + ListVector::Reserve(collection, new_capacity); + } + static inline void SetSize(Vector &collection, const idx_t new_size) { + ListVector::SetListSize(collection, new_size); + } +}; + +template +static void TupleDataCollectionWithinCollectionGather(const TupleDataLayout &layout, Vector &heap_locations, + const idx_t list_size_before, const SelectionVector &scan_sel, + const idx_t scan_count, Vector &target, + const SelectionVector &target_sel, Vector &list_vector, + const vector &child_functions) { // Source auto source_heap_locations = FlatVector::GetData(heap_locations); auto &source_heap_validity = FlatVector::Validity(heap_locations); + // Setup + CollectionVector::Setup(target); + // Target auto target_list_entries = FlatVector::GetData(target); auto &target_validity = FlatVector::Validity(target); - const auto child_list_size_before = ListVector::GetListSize(target); + const auto child_list_size_before = CollectionVector::GetSizeBefore(target); // List parent const auto list_entries = FlatVector::GetData(list_vector); @@ -1169,77 +1443,125 @@ static void TupleDataListWithinListGather(const TupleDataLayout &layout, Vector target_offset += list_length; } - ListVector::Reserve(target, target_child_offset); - ListVector::SetListSize(target, target_child_offset); + + CollectionVector::Reserve(target, target_child_offset); + CollectionVector::SetSize(target, target_child_offset); // Recurse D_ASSERT(child_functions.size() == 1); const auto &child_function = child_functions[0]; child_function.function(layout, heap_locations, child_list_size_before, scan_sel, scan_count, - ListVector::GetEntry(target), target_sel, combined_list_vector, + COLLECTION::GetEntry(target), target_sel, combined_list_vector, child_function.child_functions); } +//------------------------------------------------------------------------------ +// Get Gather Function +//------------------------------------------------------------------------------ + template -tuple_data_gather_function_t TupleDataGetGatherFunction(bool within_list) { - return within_list ? TupleDataTemplatedWithinListGather : TupleDataTemplatedGather; +tuple_data_gather_function_t TupleDataGetGatherFunction(WithinCollection within_collection) { + switch (within_collection) { + case WithinCollection::NO: + return TupleDataTemplatedGather; + case WithinCollection::LIST: + case WithinCollection::ARRAY: + return TupleDataTemplatedWithinCollectionGather; + default: + throw NotImplementedException("Unimplemented collection type"); + } } -TupleDataGatherFunction TupleDataCollection::GetGatherFunction(const LogicalType &type, bool within_list) { +TupleDataGatherFunction TupleDataCollection::GetGatherFunction(const LogicalType &type, + WithinCollection within_collection) { TupleDataGatherFunction result; switch (type.InternalType()) { case PhysicalType::BOOL: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::INT8: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::INT16: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::INT32: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::INT64: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::INT128: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::UINT8: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::UINT16: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::UINT32: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::UINT64: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::FLOAT: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::DOUBLE: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::INTERVAL: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::VARCHAR: - result.function = TupleDataGetGatherFunction(within_list); + result.function = TupleDataGetGatherFunction(within_collection); break; case PhysicalType::STRUCT: { - result.function = within_list ? TupleDataStructWithinListGather : TupleDataStructGather; + switch (within_collection) { + case WithinCollection::NO: + result.function = TupleDataStructGather; + break; + case WithinCollection::LIST: + case WithinCollection::ARRAY: + result.function = TupleDataStructWithinCollectionGather; + break; + default: + throw NotImplementedException("Unimplemented collection type"); + } for (const auto &child_type : StructType::GetChildTypes(type)) { - result.child_functions.push_back(GetGatherFunction(child_type.second, within_list)); + result.child_functions.push_back(GetGatherFunction(child_type.second, within_collection)); } break; } case PhysicalType::LIST: - result.function = within_list ? TupleDataListWithinListGather : TupleDataListGather; - result.child_functions.push_back(GetGatherFunction(ListType::GetChildType(type), true)); + switch (within_collection) { + case WithinCollection::NO: + result.function = TupleDataListGather; + break; + case WithinCollection::LIST: + case WithinCollection::ARRAY: + result.function = TupleDataCollectionWithinCollectionGather; + break; + default: + throw NotImplementedException("Unimplemented collection type"); + } + result.child_functions.push_back(GetGatherFunction(ListType::GetChildType(type), WithinCollection::LIST)); + break; + case PhysicalType::ARRAY: + switch (within_collection) { + case WithinCollection::NO: + result.function = TupleDataArrayGather; + break; + case WithinCollection::LIST: + case WithinCollection::ARRAY: + result.function = TupleDataCollectionWithinCollectionGather; + break; + default: + throw NotImplementedException("Unimplemented collection type"); + } + result.child_functions.push_back(GetGatherFunction(ArrayType::GetChildType(type), WithinCollection::ARRAY)); break; default: throw InternalException("Unsupported type for TupleDataCollection::GetGatherFunction"); diff --git a/src/duckdb/src/common/types/validity_mask.cpp b/src/duckdb/src/common/types/validity_mask.cpp index 5ec42b77f..2cf28ce3b 100644 --- a/src/duckdb/src/common/types/validity_mask.cpp +++ b/src/duckdb/src/common/types/validity_mask.cpp @@ -53,6 +53,7 @@ string ValidityMask::ToString(idx_t count) const { void ValidityMask::Resize(idx_t old_size, idx_t new_size) { D_ASSERT(new_size >= old_size); + target_count = new_size; if (validity_mask) { auto new_size_count = EntryCount(new_size); auto old_size_count = EntryCount(old_size); @@ -67,10 +68,15 @@ void ValidityMask::Resize(idx_t old_size, idx_t new_size) { validity_data = std::move(new_validity_data); validity_mask = validity_data->owned_data.get(); } else { + // TODO: We shouldn't have to initialize here, just update the target count Initialize(new_size); } } +idx_t ValidityMask::TargetCount() { + return target_count; +} + void ValidityMask::Slice(const ValidityMask &other, idx_t source_offset, idx_t count) { if (other.AllValid()) { validity_mask = nullptr; @@ -91,6 +97,7 @@ bool ValidityMask::IsAligned(idx_t count) { } void ValidityMask::SliceInPlace(const ValidityMask &other, idx_t target_offset, idx_t source_offset, idx_t count) { + EnsureWritable(); if (IsAligned(source_offset) && IsAligned(target_offset)) { auto target_validity = GetData(); auto source_validity = other.GetData(); diff --git a/src/duckdb/src/common/types/value.cpp b/src/duckdb/src/common/types/value.cpp index 22a75098e..83f45ef48 100644 --- a/src/duckdb/src/common/types/value.cpp +++ b/src/duckdb/src/common/types/value.cpp @@ -741,6 +741,41 @@ Value Value::EMPTYLIST(const LogicalType &child_type) { return result; } +Value Value::ARRAY(vector values) { + if (values.empty()) { + throw InternalException("Value::ARRAY without providing a child-type requires a non-empty list of values. Use " + "Value::ARRAY(child_type, list) instead."); + } +#ifdef DEBUG + for (idx_t i = 1; i < values.size(); i++) { + D_ASSERT(values[i].type() == values[0].type()); + } +#endif + Value result; + result.type_ = LogicalType::ARRAY(values[0].type(), values.size()); + result.value_info_ = make_shared(std::move(values)); + result.is_null = false; + return result; +} + +Value Value::ARRAY(const LogicalType &child_type, vector values) { + if (values.empty()) { + return Value::EMPTYARRAY(child_type, 0); + } + for (auto &val : values) { + val = val.DefaultCastAs(child_type); + } + return Value::ARRAY(std::move(values)); +} + +Value Value::EMPTYARRAY(const LogicalType &child_type, uint32_t size) { + Value result; + result.type_ = LogicalType::ARRAY(child_type, size); + result.value_info_ = make_shared(); + result.is_null = false; + return result; +} + Value Value::BLOB(const_data_ptr_t data, idx_t len) { Value result(LogicalType::BLOB); result.is_null = false; @@ -1337,7 +1372,7 @@ string Value::ToSQLString() const { string ret = "{"; auto &child_types = StructType::GetChildTypes(type_); auto &struct_values = StructValue::GetChildren(*this); - for (size_t i = 0; i < struct_values.size(); i++) { + for (idx_t i = 0; i < struct_values.size(); i++) { auto &name = child_types[i].first; auto &child = struct_values[i]; ret += "'" + name + "': " + child.ToSQLString(); @@ -1367,7 +1402,7 @@ string Value::ToSQLString() const { case LogicalTypeId::LIST: { string ret = "["; auto &list_values = ListValue::GetChildren(*this); - for (size_t i = 0; i < list_values.size(); i++) { + for (idx_t i = 0; i < list_values.size(); i++) { auto &child = list_values[i]; ret += child.ToSQLString(); if (i < list_values.size() - 1) { @@ -1377,6 +1412,19 @@ string Value::ToSQLString() const { ret += "]"; return ret; } + case LogicalTypeId::ARRAY: { + string ret = "["; + auto &array_values = ArrayValue::GetChildren(*this); + for (idx_t i = 0; i < array_values.size(); i++) { + auto &child = array_values[i]; + ret += child.ToSQLString(); + if (i < array_values.size() - 1) { + ret += ", "; + } + } + ret += "]"; + return ret; + } default: return ToString(); } @@ -1476,6 +1524,15 @@ const vector &ListValue::GetChildren(const Value &value) { return value.value_info_->Get().GetValues(); } +const vector &ArrayValue::GetChildren(const Value &value) { + if (value.is_null) { + throw InternalException("Calling ArrayValue::GetChildren on a NULL value"); + } + D_ASSERT(value.type().InternalType() == PhysicalType::ARRAY); + D_ASSERT(value.value_info_); + return value.value_info_->Get().GetValues(); +} + const Value &UnionValue::GetValue(const Value &value) { D_ASSERT(value.type().id() == LogicalTypeId::UNION); auto &children = StructValue::GetChildren(value); @@ -1716,6 +1773,12 @@ void Value::Serialize(Serializer &serializer) const { serializer.WriteProperty(100, "children", children); }); } break; + case PhysicalType::ARRAY: { + serializer.WriteObject(102, "value", [&](Serializer &serializer) { + auto &children = ArrayValue::GetChildren(*this); + serializer.WriteProperty(100, "children", children); + }); + } break; default: throw NotImplementedException("Unimplemented type for Serialize"); } @@ -1792,6 +1855,12 @@ Value Value::Deserialize(Deserializer &deserializer) { new_value.value_info_ = make_shared(children); }); } break; + case PhysicalType::ARRAY: { + deserializer.ReadObject(102, "value", [&](Deserializer &obj) { + auto children = obj.ReadProperty>(100, "children"); + new_value.value_info_ = make_shared(children); + }); + } break; default: throw NotImplementedException("Unimplemented type for Deserialize"); } diff --git a/src/duckdb/src/common/types/vector.cpp b/src/duckdb/src/common/types/vector.cpp index 03af4fb90..eb2494f68 100644 --- a/src/duckdb/src/common/types/vector.cpp +++ b/src/duckdb/src/common/types/vector.cpp @@ -91,6 +91,10 @@ void Vector::Reference(const Value &value) { auxiliary = shared_ptr(list_buffer.release()); data = buffer->GetData(); SetValue(0, value); + } else if (internal_type == PhysicalType::ARRAY) { + auto array_buffer = make_uniq(value.type()); + auxiliary = shared_ptr(array_buffer.release()); + SetValue(0, value); } else { auxiliary.reset(); data = buffer->GetData(); @@ -141,6 +145,14 @@ void Vector::Slice(Vector &other, idx_t offset, idx_t end) { } new_vector.validity.Slice(other.validity, offset, end - offset); Reference(new_vector); + } else if (internal_type == PhysicalType::ARRAY) { + Vector new_vector(GetType()); + auto &child_vec = ArrayVector::GetEntry(new_vector); + auto &other_child_vec = ArrayVector::GetEntry(other); + D_ASSERT(ArrayType::GetSize(GetType()) == ArrayType::GetSize(other.GetType())); + child_vec.Slice(other_child_vec, offset, end); + new_vector.validity.Slice(other.validity, offset, end - offset); + Reference(new_vector); } else { Reference(other); if (offset > 0) { @@ -223,6 +235,9 @@ void Vector::Initialize(bool zero_data, idx_t capacity) { } else if (internal_type == PhysicalType::LIST) { auto list_buffer = make_uniq(type, capacity); auxiliary = shared_ptr(list_buffer.release()); + } else if (internal_type == PhysicalType::ARRAY) { + auto array_buffer = make_uniq(type, capacity); + auxiliary = shared_ptr(array_buffer.release()); } auto type_size = GetTypeIdSize(internal_type); if (type_size > 0) { @@ -232,8 +247,9 @@ void Vector::Initialize(bool zero_data, idx_t capacity) { memset(data, 0, capacity * type_size); } } - if (capacity > STANDARD_VECTOR_SIZE) { - validity.Resize(STANDARD_VECTOR_SIZE, capacity); + + if (capacity > validity.TargetCount()) { + validity.Resize(validity.TargetCount(), capacity); } } @@ -243,12 +259,15 @@ struct DataArrays { optional_ptr buffer; idx_t type_size; bool is_nested; - DataArrays(Vector &vec, data_ptr_t data, optional_ptr buffer, idx_t type_size, bool is_nested) - : vec(vec), data(data), buffer(buffer), type_size(type_size), is_nested(is_nested) { + idx_t nested_multiplier; + DataArrays(Vector &vec, data_ptr_t data, optional_ptr buffer, idx_t type_size, bool is_nested, + idx_t nested_multiplier = 1) + : vec(vec), data(data), buffer(buffer), type_size(type_size), is_nested(is_nested), + nested_multiplier(nested_multiplier) { } }; -void FindChildren(vector &to_resize, VectorBuffer &auxiliary) { +void FindChildren(vector &to_resize, VectorBuffer &auxiliary, idx_t current_multiplier) { if (auxiliary.GetBufferType() == VectorBufferType::LIST_BUFFER) { auto &buffer = auxiliary.Cast(); auto &child = buffer.GetChild(); @@ -258,7 +277,7 @@ void FindChildren(vector &to_resize, VectorBuffer &auxiliary) { DataArrays arrays(child, data, child.GetBuffer().get(), GetTypeIdSize(child.GetType().InternalType()), true); to_resize.emplace_back(arrays); - FindChildren(to_resize, *child.GetAuxiliary()); + FindChildren(to_resize, *child.GetAuxiliary(), current_multiplier); } else { DataArrays arrays(child, data, child.GetBuffer().get(), GetTypeIdSize(child.GetType().InternalType()), false); @@ -274,13 +293,33 @@ void FindChildren(vector &to_resize, VectorBuffer &auxiliary) { DataArrays arrays(*child, data, child->GetBuffer().get(), GetTypeIdSize(child->GetType().InternalType()), true); to_resize.emplace_back(arrays); - FindChildren(to_resize, *child->GetAuxiliary()); + FindChildren(to_resize, *child->GetAuxiliary(), current_multiplier); } else { DataArrays arrays(*child, data, child->GetBuffer().get(), GetTypeIdSize(child->GetType().InternalType()), false); to_resize.emplace_back(arrays); } } + } else if (auxiliary.GetBufferType() == VectorBufferType::ARRAY_BUFFER) { + auto &buffer = auxiliary.Cast(); + auto array_size = buffer.GetArraySize(); + auto &child = buffer.GetChild(); + auto data = child.GetData(); + if (!data) { + //! Nested type + DataArrays arrays(child, data, child.GetBuffer().get(), GetTypeIdSize(child.GetType().InternalType()), true, + current_multiplier); + to_resize.emplace_back(arrays); + + // The child vectors of ArrayTypes always have to be (size * array_size), so we need to multiply the + // multiplier by the array size + auto new_multiplier = current_multiplier * array_size; + FindChildren(to_resize, *child.GetAuxiliary(), new_multiplier); + } else { + DataArrays arrays(child, data, child.GetBuffer().get(), GetTypeIdSize(child.GetType().InternalType()), + false, current_multiplier); + to_resize.emplace_back(arrays); + } } } void Vector::Resize(idx_t cur_size, idx_t new_size) { @@ -292,19 +331,25 @@ void Vector::Resize(idx_t cur_size, idx_t new_size) { //! this is a nested structure DataArrays arrays(*this, data, buffer.get(), GetTypeIdSize(GetType().InternalType()), true); to_resize.emplace_back(arrays); - FindChildren(to_resize, *auxiliary); + + // The child vectors of ArrayTypes always have to be (size * array_size), so we need to multiply the + // resize amount by the array size recursively for every nested array. + auto start_multiplier = GetType().id() == LogicalTypeId::ARRAY ? ArrayType::GetSize(GetType()) : 1; + FindChildren(to_resize, *auxiliary, start_multiplier); } else { DataArrays arrays(*this, data, buffer.get(), GetTypeIdSize(GetType().InternalType()), false); to_resize.emplace_back(arrays); } for (auto &data_to_resize : to_resize) { if (!data_to_resize.is_nested) { - auto new_data = make_unsafe_uniq_array(new_size * data_to_resize.type_size); - memcpy(new_data.get(), data_to_resize.data, cur_size * data_to_resize.type_size * sizeof(data_t)); + auto new_data = + make_unsafe_uniq_array(new_size * data_to_resize.type_size * data_to_resize.nested_multiplier); + memcpy(new_data.get(), data_to_resize.data, + cur_size * data_to_resize.type_size * data_to_resize.nested_multiplier * sizeof(data_t)); data_to_resize.buffer->SetData(std::move(new_data)); data_to_resize.vec.data = data_to_resize.buffer->GetData(); } - data_to_resize.vec.validity.Resize(cur_size, new_size); + data_to_resize.vec.validity.Resize(cur_size, new_size * data_to_resize.nested_multiplier); } } @@ -323,13 +368,14 @@ void Vector::SetValue(idx_t index, const Value &val) { validity.EnsureWritable(); validity.Set(index, !val.IsNull()); - if (val.IsNull() && GetType().InternalType() != PhysicalType::STRUCT) { - // for structs we still need to set the child-entries to NULL + auto physical_type = GetType().InternalType(); + if (val.IsNull() && physical_type != PhysicalType::STRUCT && physical_type != PhysicalType::ARRAY) { + // for structs and arrays we still need to set the child-entries to NULL // so we do not bail out yet return; } - switch (GetType().InternalType()) { + switch (physical_type) { case PhysicalType::BOOL: reinterpret_cast(data)[index] = val.GetValueUnsafe(); break; @@ -406,6 +452,21 @@ void Vector::SetValue(idx_t index, const Value &val) { entry.offset = offset; break; } + case PhysicalType::ARRAY: { + auto array_size = ArrayType::GetSize(GetType()); + auto &child = ArrayVector::GetEntry(*this); + if (val.IsNull()) { + for (idx_t i = 0; i < array_size; i++) { + child.SetValue(index * array_size + i, Value()); + } + } else { + auto &val_children = ArrayValue::GetChildren(val); + for (idx_t i = 0; i < array_size; i++) { + child.SetValue(index * array_size + i, val_children[i]); + } + } + break; + } default: throw InternalException("Unimplemented type for Vector::SetValue"); } @@ -585,6 +646,16 @@ Value Vector::GetValueInternal(const Vector &v_p, idx_t index_p) { } return Value::LIST(ListType::GetChildType(type), std::move(children)); } + case LogicalTypeId::ARRAY: { + auto stride = ArrayType::GetSize(type); + auto offset = index * stride; + auto &child_vec = ArrayVector::GetEntry(*vector); + duckdb::vector children; + for (idx_t i = offset; i < offset + stride; i++) { + children.push_back(child_vec.GetValue(i)); + } + return Value::ARRAY(std::move(children)); + } default: throw InternalException("Unimplemented type for value access"); } @@ -793,6 +864,45 @@ void Vector::Flatten(idx_t count) { TemplatedFlattenConstantVector(data, old_data, count); break; } + case PhysicalType::ARRAY: { + auto &child = ArrayVector::GetEntry(*this); + auto array_size = ArrayType::GetSize(GetType()); + + auto flattened_buffer = make_uniq(GetType(), count); + auto &new_child = flattened_buffer->GetChild(); + + // Make sure to initialize a validity mask for the new child vector with the correct size + if (!child.validity.AllValid()) { + new_child.validity.Initialize(array_size * count); + } + + // Now we need to "unpack" the child vector. + // Basically, do this: + // + // | a1 | | 1 | | a1 | | 1 | + // | 2 | | a2 | | 2 | + // => .. | 1 | + // | 2 | + // ... + + // Create a selection vector + SelectionVector sel(count * array_size); + for (idx_t array_idx = 0; array_idx < count; array_idx++) { + for (idx_t elem_idx = 0; elem_idx < array_size; elem_idx++) { + auto position = array_idx * array_size + elem_idx; + // Broadcast the validity + if (FlatVector::IsNull(child, elem_idx)) { + FlatVector::SetNull(new_child, position, true); + } + sel.set_index(position, elem_idx); + } + } + + // Copy over the data to the new buffer + VectorOperations::Copy(child, new_child, sel, count * array_size, 0, 0); + auxiliary = shared_ptr(flattened_buffer.release()); + + } break; case PhysicalType::STRUCT: { auto normalified_buffer = make_uniq(); @@ -894,6 +1004,7 @@ void Vector::ToUnifiedFormat(idx_t count, UnifiedVectorFormat &format) { void Vector::RecursiveToUnifiedFormat(Vector &input, idx_t count, RecursiveUnifiedVectorFormat &data) { input.ToUnifiedFormat(count, data.unified); + data.logical_type = input.GetType(); if (input.GetType().InternalType() == PhysicalType::LIST) { auto &child = ListVector::GetEntry(input); @@ -901,6 +1012,13 @@ void Vector::RecursiveToUnifiedFormat(Vector &input, idx_t count, RecursiveUnifi data.children.emplace_back(); Vector::RecursiveToUnifiedFormat(child, child_count, data.children.back()); + } else if (input.GetType().InternalType() == PhysicalType::ARRAY) { + auto &child = ArrayVector::GetEntry(input); + auto array_size = ArrayType::GetSize(input.GetType()); + auto child_count = count * array_size; + data.children.emplace_back(); + Vector::RecursiveToUnifiedFormat(child, child_count, data.children.back()); + } else if (input.GetType().InternalType() == PhysicalType::STRUCT) { auto &children = StructVector::GetEntries(input); for (idx_t i = 0; i < children.size(); i++) { @@ -933,6 +1051,7 @@ void Vector::Serialize(Serializer &serializer, idx_t count) { serializer.WriteProperty(100, "all_valid", all_valid); if (all_valid) { ValidityMask flat_mask(count); + flat_mask.Initialize(); for (idx_t i = 0; i < count; ++i) { auto row_idx = vdata.sel->get_index(i); flat_mask.Set(i, vdata.validity.RowIsValid(row_idx)); @@ -991,6 +1110,15 @@ void Vector::Serialize(Serializer &serializer, idx_t count) { serializer.WriteObject(106, "child", [&](Serializer &object) { child.Serialize(object, list_size); }); break; } + case PhysicalType::ARRAY: { + Flatten(count); + auto &child = ArrayVector::GetEntry(*this); + auto array_size = ArrayType::GetSize(type); + auto child_size = array_size * count; + serializer.WriteProperty(103, "array_size", array_size); + serializer.WriteObject(104, "child", [&](Serializer &object) { child.Serialize(object, child_size); }); + break; + } default: throw InternalException("Unimplemented variable width type for Vector::Serialize!"); } @@ -1057,6 +1185,14 @@ void Vector::Deserialize(Deserializer &deserializer, idx_t count) { }); break; } + case PhysicalType::ARRAY: { + auto array_size = deserializer.ReadProperty(103, "array_size"); + deserializer.ReadObject(104, "child", [&](Deserializer &obj) { + auto &child = ArrayVector::GetEntry(*this); + child.Deserialize(obj, array_size * count); + }); + break; + } default: throw InternalException("Unimplemented variable width type for Vector::Deserialize!"); } @@ -1065,11 +1201,12 @@ void Vector::Deserialize(Deserializer &deserializer, idx_t count) { void Vector::SetVectorType(VectorType vector_type_p) { this->vector_type = vector_type_p; - if (TypeIsConstantSize(GetType().InternalType()) && + auto physical_type = GetType().InternalType(); + if (TypeIsConstantSize(physical_type) && (GetVectorType() == VectorType::CONSTANT_VECTOR || GetVectorType() == VectorType::FLAT_VECTOR)) { auxiliary.reset(); } - if (vector_type == VectorType::CONSTANT_VECTOR && GetType().InternalType() == PhysicalType::STRUCT) { + if (vector_type == VectorType::CONSTANT_VECTOR && physical_type == PhysicalType::STRUCT) { auto &entries = StructVector::GetEntries(*this); for (auto &entry : entries) { entry->SetVectorType(vector_type); @@ -1204,6 +1341,53 @@ void Vector::Verify(Vector &vector_p, const SelectionVector &sel_p, idx_t count) } } + if (type.InternalType() == PhysicalType::ARRAY) { + // Arrays have the following invariants + // 1. if the array vector is a CONSTANT_VECTOR + // 1.1 The child vector is a FLAT_VECTOR with count = array_size + // 1.2 OR The child vector is a CONSTANT_VECTOR and must be NULL + // 1.3 OR The child vector is a CONSTANT_VECTOR and array_size = 1 + // 2. if the array vector is a FLAT_VECTOR, the child vector is a FLAT_VECTOR + // 2.2 the count of the child vector is array_size * (parent)count + + auto &child = ArrayVector::GetEntry(*vector); + auto array_size = ArrayType::GetSize(type); + + if (child.GetVectorType() == VectorType::CONSTANT_VECTOR) { + D_ASSERT(ConstantVector::IsNull(child)); + } else { + D_ASSERT(child.GetVectorType() == VectorType::FLAT_VECTOR); + } + + if (vtype == VectorType::CONSTANT_VECTOR) { + if (!ConstantVector::IsNull(*vector)) { + child.Verify(array_size); + } + } else if (vtype == VectorType::FLAT_VECTOR) { + // Flat vector case + auto &validity = FlatVector::Validity(*vector); + idx_t selected_child_count = 0; + for (idx_t i = 0; i < count; i++) { + auto oidx = sel->get_index(i); + if (validity.RowIsValid(oidx)) { + selected_child_count += array_size; + } + } + + SelectionVector child_sel(selected_child_count); + idx_t child_count = 0; + for (idx_t i = 0; i < count; i++) { + auto oidx = sel->get_index(i); + if (validity.RowIsValid(oidx)) { + for (idx_t j = 0; j < array_size; j++) { + child_sel.set_index(child_count++, oidx * array_size + j); + } + } + } + Vector::Verify(child, child_sel, child_count); + } + } + if (type.InternalType() == PhysicalType::STRUCT) { auto &child_types = StructType::GetChildTypes(type); D_ASSERT(!child_types.empty()); @@ -1318,11 +1502,24 @@ void Vector::Verify(idx_t count) { void FlatVector::SetNull(Vector &vector, idx_t idx, bool is_null) { D_ASSERT(vector.GetVectorType() == VectorType::FLAT_VECTOR); vector.validity.Set(idx, !is_null); - if (is_null && vector.GetType().InternalType() == PhysicalType::STRUCT) { - // set all child entries to null as well - auto &entries = StructVector::GetEntries(vector); - for (auto &entry : entries) { - FlatVector::SetNull(*entry, idx, is_null); + if (is_null) { + auto type = vector.GetType(); + auto internal_type = type.InternalType(); + if (internal_type == PhysicalType::STRUCT) { + // set all child entries to null as well + auto &entries = StructVector::GetEntries(vector); + for (auto &entry : entries) { + FlatVector::SetNull(*entry, idx, is_null); + } + } else if (internal_type == PhysicalType::ARRAY) { + // set the child element in the array to null as well + auto &child = ArrayVector::GetEntry(vector); + D_ASSERT(child.GetVectorType() == VectorType::FLAT_VECTOR); + auto array_size = ArrayType::GetSize(type); + auto child_offset = idx * array_size; + for (idx_t i = 0; i < array_size; i++) { + FlatVector::SetNull(child, child_offset + i, is_null); + } } } } @@ -1333,12 +1530,29 @@ void FlatVector::SetNull(Vector &vector, idx_t idx, bool is_null) { void ConstantVector::SetNull(Vector &vector, bool is_null) { D_ASSERT(vector.GetVectorType() == VectorType::CONSTANT_VECTOR); vector.validity.Set(0, !is_null); - if (is_null && vector.GetType().InternalType() == PhysicalType::STRUCT) { - // set all child entries to null as well - auto &entries = StructVector::GetEntries(vector); - for (auto &entry : entries) { - entry->SetVectorType(VectorType::CONSTANT_VECTOR); - ConstantVector::SetNull(*entry, is_null); + if (is_null) { + auto &type = vector.GetType(); + auto internal_type = type.InternalType(); + if (internal_type == PhysicalType::STRUCT) { + // set all child entries to null as well + auto &entries = StructVector::GetEntries(vector); + for (auto &entry : entries) { + entry->SetVectorType(VectorType::CONSTANT_VECTOR); + ConstantVector::SetNull(*entry, is_null); + } + } else if (internal_type == PhysicalType::ARRAY) { + auto &child = ArrayVector::GetEntry(vector); + D_ASSERT(child.GetVectorType() == VectorType::CONSTANT_VECTOR || + child.GetVectorType() == VectorType::FLAT_VECTOR); + auto array_size = ArrayType::GetSize(type); + if (child.GetVectorType() == VectorType::CONSTANT_VECTOR) { + D_ASSERT(array_size == 1); + ConstantVector::SetNull(child, is_null); + } else { + for (idx_t i = 0; i < array_size; i++) { + FlatVector::SetNull(child, i, is_null); + } + } } } } @@ -1386,6 +1600,35 @@ void ConstantVector::Reference(Vector &vector, Vector &source, idx_t position, i vector.SetVectorType(VectorType::CONSTANT_VECTOR); break; } + case PhysicalType::ARRAY: { + UnifiedVectorFormat vdata; + source.ToUnifiedFormat(count, vdata); + + if (!vdata.validity.RowIsValid(position)) { + // list is null: create null value + Value null_value(source_type); + vector.Reference(null_value); + break; + } + + // Reference the child vector + auto &target_child = ArrayVector::GetEntry(vector); + auto &source_child = ArrayVector::GetEntry(source); + target_child.Reference(source_child); + + // Only take the element at the given position + auto array_size = ArrayType::GetSize(source_type); + SelectionVector sel(array_size); + for (idx_t i = 0; i < array_size; i++) { + sel.set_index(i, array_size * position + i); + } + target_child.Slice(sel, array_size); + target_child.Flatten(array_size); // since its constant we only have to flatten this much + + vector.SetVectorType(VectorType::CONSTANT_VECTOR); + vector.validity.Set(0, true); + break; + } case PhysicalType::STRUCT: { UnifiedVectorFormat vdata; source.ToUnifiedFormat(count, vdata); @@ -2048,4 +2291,43 @@ UnionInvalidReason UnionVector::CheckUnionValidity(Vector &vector_p, idx_t count return UnionInvalidReason::VALID; } +//===--------------------------------------------------------------------===// +// ArrayVector +//===--------------------------------------------------------------------===// +const Vector &ArrayVector::GetEntry(const Vector &vector) { + D_ASSERT(vector.GetType().id() == LogicalTypeId::ARRAY); + if (vector.GetVectorType() == VectorType::DICTIONARY_VECTOR) { + auto &child = DictionaryVector::Child(vector); + return ArrayVector::GetEntry(child); + } + D_ASSERT(vector.GetVectorType() == VectorType::FLAT_VECTOR || + vector.GetVectorType() == VectorType::CONSTANT_VECTOR); + D_ASSERT(vector.auxiliary); + D_ASSERT(vector.auxiliary->GetBufferType() == VectorBufferType::ARRAY_BUFFER); + return vector.auxiliary->Cast().GetChild(); +} + +Vector &ArrayVector::GetEntry(Vector &vector) { + const Vector &cvector = vector; + return const_cast(ArrayVector::GetEntry(cvector)); +} + +idx_t ArrayVector::GetTotalSize(const Vector &vector) { + D_ASSERT(vector.GetType().id() == LogicalTypeId::ARRAY); + D_ASSERT(vector.auxiliary); + if (vector.GetVectorType() == VectorType::DICTIONARY_VECTOR) { + auto &child = DictionaryVector::Child(vector); + return ArrayVector::GetTotalSize(child); + } + return vector.auxiliary->Cast().GetChildSize(); +} + +void ArrayVector::AllocateDummyListEntries(Vector &vector) { + D_ASSERT(vector.GetType().InternalType() == PhysicalType::ARRAY); + auto array_size = ArrayType::GetSize(vector.GetType()); + auto array_count = ArrayVector::GetTotalSize(vector) / array_size; + vector.buffer = VectorBuffer::CreateStandardVector(LogicalType::HUGEINT, array_count); + vector.data = vector.buffer->GetData(); +} + } // namespace duckdb diff --git a/src/duckdb/src/common/types/vector_buffer.cpp b/src/duckdb/src/common/types/vector_buffer.cpp index fcd7066f9..1d999abf1 100644 --- a/src/duckdb/src/common/types/vector_buffer.cpp +++ b/src/duckdb/src/common/types/vector_buffer.cpp @@ -107,6 +107,35 @@ void VectorListBuffer::SetSize(idx_t new_size) { VectorListBuffer::~VectorListBuffer() { } +VectorArrayBuffer::VectorArrayBuffer(unique_ptr child_vector, idx_t array_size, idx_t initial_capacity) + : VectorBuffer(VectorBufferType::ARRAY_BUFFER), child(std::move(child_vector)), array_size(array_size), + size(initial_capacity) { + D_ASSERT(array_size != 0); +} + +VectorArrayBuffer::VectorArrayBuffer(const LogicalType &array, idx_t initial) + : VectorBuffer(VectorBufferType::ARRAY_BUFFER), + child(make_uniq(ArrayType::GetChildType(array), initial * ArrayType::GetSize(array))), + array_size(ArrayType::GetSize(array)), size(initial) { + // initialize the child array with (array_size * size) ^ + D_ASSERT(!ArrayType::IsAnySize(array)); +} + +VectorArrayBuffer::~VectorArrayBuffer() { +} + +Vector &VectorArrayBuffer::GetChild() { + return *child; +} + +idx_t VectorArrayBuffer::GetArraySize() { + return array_size; +} + +idx_t VectorArrayBuffer::GetChildSize() { + return size * array_size; +} + ManagedVectorBuffer::ManagedVectorBuffer(BufferHandle handle) : VectorBuffer(VectorBufferType::MANAGED_BUFFER), handle(std::move(handle)) { } diff --git a/src/duckdb/src/common/types/vector_cache.cpp b/src/duckdb/src/common/types/vector_cache.cpp index 12f8827e6..d7f3121a6 100644 --- a/src/duckdb/src/common/types/vector_cache.cpp +++ b/src/duckdb/src/common/types/vector_cache.cpp @@ -21,6 +21,14 @@ class VectorCacheBuffer : public VectorBuffer { auxiliary = make_shared(std::move(child_vector)); break; } + case PhysicalType::ARRAY: { + auto &child_type = ArrayType::GetChildType(type); + auto array_size = ArrayType::GetSize(type); + child_caches.push_back(make_buffer(allocator, child_type, array_size * capacity)); + auto child_vector = make_uniq(child_type, true, false, array_size * capacity); + auxiliary = make_shared(std::move(child_vector), array_size, capacity); + break; + } case PhysicalType::STRUCT: { auto &child_types = StructType::GetChildTypes(type); for (auto &child_type : child_types) { @@ -58,6 +66,23 @@ class VectorCacheBuffer : public VectorBuffer { child_cache.ResetFromCache(list_child, child_caches[0]); break; } + case PhysicalType::ARRAY: { + // fixed size list does not have own data + result.data = nullptr; + // reinitialize the VectorArrayBuffer + // auxiliary->SetAuxiliaryData(nullptr); + AssignSharedPointer(result.auxiliary, auxiliary); + + // propagate through child + auto &child_cache = child_caches[0]->Cast(); + auto &array_child = result.auxiliary->Cast().GetChild(); + child_cache.ResetFromCache(array_child, child_caches[0]); + + // Ensure the child validity is (will be) large enough, even if its not initialized. + auto validity_target_size = array_child.validity.TargetCount(); + array_child.validity.Resize(validity_target_size, std::max(validity_target_size, child_cache.capacity)); + break; + } case PhysicalType::STRUCT: { // struct does not have data result.data = nullptr; diff --git a/src/duckdb/src/common/value_operations/comparison_operations.cpp b/src/duckdb/src/common/value_operations/comparison_operations.cpp index 3680ff810..5ba5824e2 100644 --- a/src/duckdb/src/common/value_operations/comparison_operations.cpp +++ b/src/duckdb/src/common/value_operations/comparison_operations.cpp @@ -170,6 +170,23 @@ static bool TemplatedBooleanOperation(const Value &left, const Value &right) { } return false; } + case PhysicalType::ARRAY: { + auto &left_children = ArrayValue::GetChildren(left); + auto &right_children = ArrayValue::GetChildren(right); + + // Should be enforced by the type + D_ASSERT(left_children.size() == right_children.size()); + + for (idx_t i = 0; i < left_children.size(); i++) { + if (ValuePositionComparator::Definite(left_children[i], right_children[i])) { + return true; + } + if (!ValuePositionComparator::Possible(left_children[i], right_children[i])) { + return false; + } + } + return true; + } default: throw InternalException("Unimplemented type for value comparison"); } diff --git a/src/duckdb/src/common/vector_operations/comparison_operators.cpp b/src/duckdb/src/common/vector_operations/comparison_operators.cpp index 33686f376..6c97eee68 100644 --- a/src/duckdb/src/common/vector_operations/comparison_operators.cpp +++ b/src/duckdb/src/common/vector_operations/comparison_operators.cpp @@ -251,6 +251,7 @@ struct ComparisonExecutor { break; case PhysicalType::LIST: case PhysicalType::STRUCT: + case PhysicalType::ARRAY: NestedComparisonExecutor(left, right, result, count); break; default: diff --git a/src/duckdb/src/common/vector_operations/is_distinct_from.cpp b/src/duckdb/src/common/vector_operations/is_distinct_from.cpp index 1564a3911..c14ff0c3a 100644 --- a/src/duckdb/src/common/vector_operations/is_distinct_from.cpp +++ b/src/duckdb/src/common/vector_operations/is_distinct_from.cpp @@ -671,6 +671,119 @@ static idx_t DistinctSelectList(Vector &left, Vector &right, idx_t count, const return match_count; } +static void PositionArrayCursor(SelectionVector &cursor, UnifiedVectorFormat &vdata, const idx_t pos, + const SelectionVector &slice_sel, const idx_t count, idx_t array_size) { + for (idx_t i = 0; i < count; ++i) { + const auto slice_idx = slice_sel.get_index(i); + const auto lidx = vdata.sel->get_index(slice_idx); + const auto offset = array_size * lidx; + cursor.set_index(i, offset + pos); + } +} + +template +static idx_t DistinctSelectArray(Vector &left, Vector &right, idx_t count, const SelectionVector &sel, + OptionalSelection &true_opt, OptionalSelection &false_opt) { + if (count == 0) { + return count; + } + + // FIXME: This function can probably be optimized since we know the array size is fixed for every entry. + + D_ASSERT(ArrayType::GetSize(left.GetType()) == ArrayType::GetSize(right.GetType())); + auto array_size = ArrayType::GetSize(left.GetType()); + + // Create dictionary views of the children so we can vectorise the positional comparisons. + SelectionVector lcursor(count); + SelectionVector rcursor(count); + + Vector lentry_flattened(ArrayVector::GetEntry(left)); + Vector rentry_flattened(ArrayVector::GetEntry(right)); + lentry_flattened.Flatten(ArrayVector::GetTotalSize(left)); + rentry_flattened.Flatten(ArrayVector::GetTotalSize(right)); + Vector lchild(lentry_flattened, lcursor, count); + Vector rchild(rentry_flattened, rcursor, count); + + // Get pointers to the list entries + UnifiedVectorFormat lvdata; + left.ToUnifiedFormat(count, lvdata); + + UnifiedVectorFormat rvdata; + right.ToUnifiedFormat(count, rvdata); + + // In order to reuse the comparators, we have to track what passed and failed internally. + // To do that, we need local SVs that we then merge back into the real ones after every pass. + SelectionVector slice_sel(count); + for (idx_t i = 0; i < count; ++i) { + slice_sel.set_index(i, i); + } + + SelectionVector true_sel(count); + SelectionVector false_sel(count); + + idx_t match_count = 0; + for (idx_t pos = 0; count > 0; ++pos) { + // Set up the cursors for the current position + PositionArrayCursor(lcursor, lvdata, pos, slice_sel, count, array_size); + PositionArrayCursor(rcursor, rvdata, pos, slice_sel, count, array_size); + + // Tie-break the pairs where one of the LISTs is exhausted. + idx_t true_count = 0; + idx_t false_count = 0; + idx_t maybe_count = 0; + for (idx_t i = 0; i < count; ++i) { + const auto slice_idx = slice_sel.get_index(i); + if (array_size == pos) { + const auto idx = sel.get_index(slice_idx); + if (PositionComparator::TieBreak(array_size, array_size)) { + true_opt.Append(true_count, idx); + } else { + false_opt.Append(false_count, idx); + } + } else { + true_sel.set_index(maybe_count++, slice_idx); + } + } + true_opt.Advance(true_count); + false_opt.Advance(false_count); + match_count += true_count; + + // Redensify the list cursors + if (maybe_count < count) { + count = maybe_count; + DensifyNestedSelection(true_sel, count, slice_sel); + PositionArrayCursor(lcursor, lvdata, pos, slice_sel, count, array_size); + PositionArrayCursor(rcursor, rvdata, pos, slice_sel, count, array_size); + } + + // Find everything that definitely matches + true_count = PositionComparator::Definite(lchild, rchild, slice_sel, count, &true_sel, false_sel); + if (true_count) { + false_count = count - true_count; + ExtractNestedSelection(false_count ? true_sel : slice_sel, true_count, sel, true_opt); + match_count += true_count; + + // Redensify the list cursors + count -= true_count; + DensifyNestedSelection(false_sel, count, slice_sel); + PositionArrayCursor(lcursor, lvdata, pos, slice_sel, count, array_size); + PositionArrayCursor(rcursor, rvdata, pos, slice_sel, count, array_size); + } + + // Find what might match on the next position + true_count = PositionComparator::Possible(lchild, rchild, slice_sel, count, true_sel, &false_sel); + false_count = count - true_count; + ExtractNestedSelection(true_count ? false_sel : slice_sel, false_count, sel, false_opt); + + if (false_count) { + DensifyNestedSelection(true_sel, true_count, slice_sel); + } + count = true_count; + } + + return match_count; +} + template static idx_t DistinctSelectNested(Vector &left, Vector &right, const SelectionVector *sel, const idx_t count, SelectionVector *true_sel, SelectionVector *false_sel) { @@ -700,10 +813,18 @@ static idx_t DistinctSelectNested(Vector &left, Vector &right, const SelectionVe auto unknown = DistinctSelectNotNull(l_not_null, r_not_null, count, match_count, *sel, maybe_vec, true_opt, false_opt); - if (PhysicalType::LIST == left.GetType().InternalType()) { + switch (left.GetType().InternalType()) { + case PhysicalType::LIST: match_count += DistinctSelectList(l_not_null, r_not_null, unknown, maybe_vec, true_opt, false_opt); - } else { + break; + case PhysicalType::STRUCT: match_count += DistinctSelectStruct(l_not_null, r_not_null, unknown, maybe_vec, true_opt, false_opt); + break; + case PhysicalType::ARRAY: + match_count += DistinctSelectArray(l_not_null, r_not_null, unknown, maybe_vec, true_opt, false_opt); + break; + default: + throw NotImplementedException("Unimplemented type for DISTINCT"); } // Copy the buffered selections to the output selections @@ -772,6 +893,7 @@ static void ExecuteDistinct(Vector &left, Vector &right, Vector &result, idx_t c break; case PhysicalType::LIST: case PhysicalType::STRUCT: + case PhysicalType::ARRAY: NestedDistinctExecute(left, right, result, count); break; default: @@ -813,6 +935,7 @@ static idx_t TemplatedDistinctSelectOperation(Vector &left, Vector &right, const return DistinctSelect(left, right, sel, count, true_sel, false_sel); case PhysicalType::STRUCT: case PhysicalType::LIST: + case PhysicalType::ARRAY: return DistinctSelectNested(left, right, sel, count, true_sel, false_sel); default: throw InternalException("Invalid type for distinct selection"); diff --git a/src/duckdb/src/common/vector_operations/vector_copy.cpp b/src/duckdb/src/common/vector_operations/vector_copy.cpp index 4f1870def..a2e0f523d 100644 --- a/src/duckdb/src/common/vector_operations/vector_copy.cpp +++ b/src/duckdb/src/common/vector_operations/vector_copy.cpp @@ -192,6 +192,26 @@ void VectorOperations::Copy(const Vector &source_p, Vector &target, const Select } break; } + case PhysicalType::ARRAY: { + D_ASSERT(target.GetType().InternalType() == PhysicalType::ARRAY); + D_ASSERT(ArrayType::GetSize(source->GetType()) == ArrayType::GetSize(target.GetType())); + + auto &source_child = ArrayVector::GetEntry(*source); + auto &target_child = ArrayVector::GetEntry(target); + auto array_size = ArrayType::GetSize(source->GetType()); + + // Create a selection vector for the child elements + SelectionVector child_sel(copy_count * array_size); + for (idx_t i = 0; i < copy_count; i++) { + auto source_idx = sel->get_index(source_offset + i); + for (idx_t j = 0; j < array_size; j++) { + child_sel.set_index(i * array_size + j, source_idx * array_size + j); + } + } + VectorOperations::Copy(source_child, target_child, child_sel, source_count * array_size, + source_offset * array_size, target_offset * array_size); + break; + } case PhysicalType::LIST: { D_ASSERT(target.GetType().InternalType() == PhysicalType::LIST); diff --git a/src/duckdb/src/common/vector_operations/vector_hash.cpp b/src/duckdb/src/common/vector_operations/vector_hash.cpp index f489d8f27..422529a38 100644 --- a/src/duckdb/src/common/vector_operations/vector_hash.cpp +++ b/src/duckdb/src/common/vector_operations/vector_hash.cpp @@ -177,6 +177,34 @@ static inline void ListLoopHash(Vector &input, Vector &hashes, const SelectionVe } } +template +static inline void ArrayLoopHash(Vector &input, Vector &hashes, const SelectionVector *rsel, idx_t count) { + auto hdata = FlatVector::GetData(hashes); + + UnifiedVectorFormat idata; + input.ToUnifiedFormat(count, idata); + + // Hash the children into a temporary + auto &child = ArrayVector::GetEntry(input); + auto array_size = ArrayType::GetSize(input.GetType()); + auto child_count = array_size * count; + + Vector child_hashes(LogicalType::HASH, child_count); + if (child_count > 0) { + child_hashes.Flatten(child_count); + VectorOperations::Hash(child, child_hashes, child_count); + } + auto chdata = FlatVector::GetData(child_hashes); + + // Combine hashes for every array + // TODO: Branch on FIRST_HASH and HAS_RSEL + for (idx_t i = 0; i < count; i++) { + for (idx_t j = i * array_size; j < (i + 1) * array_size; j++) { + hdata[i] = CombineHashScalar(hdata[i], chdata[j]); + } + } +} + template static inline void HashTypeSwitch(Vector &input, Vector &result, const SelectionVector *rsel, idx_t count) { D_ASSERT(result.GetType().id() == LogicalType::HASH); @@ -227,6 +255,9 @@ static inline void HashTypeSwitch(Vector &input, Vector &result, const Selection case PhysicalType::LIST: ListLoopHash(input, result, rsel, count); break; + case PhysicalType::ARRAY: + ArrayLoopHash(input, result, rsel, count); + break; default: throw InvalidTypeException(input.GetType(), "Invalid type for hash"); } @@ -360,6 +391,9 @@ static inline void CombineHashTypeSwitch(Vector &hashes, Vector &input, const Se case PhysicalType::LIST: ListLoopHash(input, hashes, rsel, count); break; + case PhysicalType::ARRAY: + ArrayLoopHash(input, hashes, rsel, count); + break; default: throw InvalidTypeException(input.GetType(), "Invalid type for hash"); } diff --git a/src/duckdb/src/core_functions/aggregate/distributive/minmax.cpp b/src/duckdb/src/core_functions/aggregate/distributive/minmax.cpp index fa087bab3..c8627ea94 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/minmax.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/minmax.cpp @@ -234,6 +234,9 @@ static bool TemplatedOptimumList(Vector &left, idx_t lidx, idx_t lcount, Vector template static bool TemplatedOptimumStruct(Vector &left, idx_t lidx, idx_t lcount, Vector &right, idx_t ridx, idx_t rcount); +template +static bool TemplatedOptimumArray(Vector &left, idx_t lidx, idx_t lcount, Vector &right, idx_t ridx, idx_t rcount); + template static bool TemplatedOptimumValue(Vector &left, idx_t lidx, idx_t lcount, Vector &right, idx_t ridx, idx_t rcount) { D_ASSERT(left.GetType() == right.GetType()); @@ -269,6 +272,8 @@ static bool TemplatedOptimumValue(Vector &left, idx_t lidx, idx_t lcount, Vector return TemplatedOptimumList(left, lidx, lcount, right, ridx, rcount); case PhysicalType::STRUCT: return TemplatedOptimumStruct(left, lidx, lcount, right, ridx, rcount); + case PhysicalType::ARRAY: + return TemplatedOptimumArray(left, lidx, lcount, right, ridx, rcount); default: throw InternalException("Invalid type for distinct comparison"); } @@ -370,6 +375,52 @@ static bool TemplatedOptimumList(Vector &left, idx_t lidx, idx_t lcount, Vector return false; } +// FIXME: We should try to unify this with TemplatedOptimumList +template +static bool TemplatedOptimumArray(Vector &left, idx_t lidx_p, idx_t lcount, Vector &right, idx_t ridx_p, idx_t rcount) { + // so map the indexes first + UnifiedVectorFormat lvdata, rvdata; + left.ToUnifiedFormat(lcount, lvdata); + right.ToUnifiedFormat(rcount, rvdata); + + idx_t lidx = lvdata.sel->get_index(lidx_p); + idx_t ridx = rvdata.sel->get_index(ridx_p); + + // DISTINCT semantics are in effect for nested types + auto lnull = !lvdata.validity.RowIsValid(lidx); + auto rnull = !rvdata.validity.RowIsValid(ridx); + if (lnull || rnull) { + return OP::Operation(0, 0, lnull, rnull); + } + + auto &lchild = ArrayVector::GetEntry(left); + auto &rchild = ArrayVector::GetEntry(right); + auto left_array_size = ArrayType::GetSize(left.GetType()); + auto right_array_size = ArrayType::GetSize(right.GetType()); + + D_ASSERT(left_array_size == right_array_size); + + auto lchild_count = lcount * left_array_size; + auto rchild_count = rcount * right_array_size; + + for (idx_t elem_idx = 0; elem_idx < left_array_size; elem_idx++) { + auto left_elem_idx = lidx * left_array_size + elem_idx; + auto right_elem_idx = ridx * right_array_size + elem_idx; + + // Strict comparisons use the OP for definite + if (TemplatedOptimumValue(lchild, left_elem_idx, lchild_count, rchild, right_elem_idx, rchild_count)) { + return true; + } + + // Strict comparisons use IS NOT DISTINCT for possible + if (!TemplatedOptimumValue(lchild, left_elem_idx, lchild_count, rchild, right_elem_idx, + rchild_count)) { + return false; + } + } + return false; +} + struct VectorMinMaxState { Vector *value; }; @@ -515,12 +566,16 @@ static AggregateFunction GetMinMaxFunction(const LogicalType &type) { template static AggregateFunction GetMinMaxOperator(const LogicalType &type) { - if (type.InternalType() == PhysicalType::VARCHAR) { + auto internal_type = type.InternalType(); + switch (internal_type) { + case PhysicalType::VARCHAR: return AggregateFunction::UnaryAggregateDestructor, string_t, string_t, OP_STRING>( type.id(), type.id()); - } else if (type.InternalType() == PhysicalType::LIST || type.InternalType() == PhysicalType::STRUCT) { + case PhysicalType::LIST: + case PhysicalType::STRUCT: + case PhysicalType::ARRAY: return GetMinMaxFunction(type); - } else { + default: return GetUnaryAggregate(type); } } diff --git a/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp b/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp index 0f81b88c9..7eaf323fc 100644 --- a/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp +++ b/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp @@ -41,21 +41,17 @@ struct ModeState { }; using Counts = unordered_map; - Counts *frequency_map; - KEY_TYPE *mode; - size_t nonzero; - bool valid; - size_t count; - - void Initialize() { - frequency_map = nullptr; - mode = nullptr; - nonzero = 0; - valid = false; - count = 0; + ModeState() { } - void Destroy() { + vector prevs; + Counts *frequency_map = nullptr; + KEY_TYPE *mode = nullptr; + size_t nonzero = 0; + bool valid = false; + size_t count = 0; + + ~ModeState() { if (frequency_map) { delete frequency_map; } @@ -119,16 +115,15 @@ struct ModeState { }; struct ModeIncluded { - inline explicit ModeIncluded(const ValidityMask &fmask_p, const ValidityMask &dmask_p, idx_t bias_p) - : fmask(fmask_p), dmask(dmask_p), bias(bias_p) { + inline explicit ModeIncluded(const ValidityMask &fmask_p, const ValidityMask &dmask_p) + : fmask(fmask_p), dmask(dmask_p) { } inline bool operator()(const idx_t &idx) const { - return fmask.RowIsValid(idx) && dmask.RowIsValid(idx - bias); + return fmask.RowIsValid(idx) && dmask.RowIsValid(idx); } const ValidityMask &fmask; const ValidityMask &dmask; - const idx_t bias; }; struct ModeAssignmentStandard { @@ -149,7 +144,7 @@ template struct ModeFunction { template static void Initialize(STATE &state) { - state.Initialize(); + new (&state) STATE(); } template @@ -207,51 +202,72 @@ struct ModeFunction { state.count += count; } + template + struct UpdateWindowState { + STATE &state; + const INPUT_TYPE *data; + ModeIncluded &included; + + inline UpdateWindowState(STATE &state, const INPUT_TYPE *data, ModeIncluded &included) + : state(state), data(data), included(included) { + } + + inline void Neither(idx_t begin, idx_t end) { + } + + inline void Left(idx_t begin, idx_t end) { + for (; begin < end; ++begin) { + if (included(begin)) { + state.ModeRm(KEY_TYPE(data[begin]), begin); + } + } + } + + inline void Right(idx_t begin, idx_t end) { + for (; begin < end; ++begin) { + if (included(begin)) { + state.ModeAdd(KEY_TYPE(data[begin]), begin); + } + } + } + + inline void Both(idx_t begin, idx_t end) { + } + }; + template static void Window(const INPUT_TYPE *data, const ValidityMask &fmask, const ValidityMask &dmask, - AggregateInputData &, STATE &state, const FrameBounds &frame, const FrameBounds &prev, - Vector &result, idx_t rid, idx_t bias) { + AggregateInputData &aggr_input_data, STATE &state, const vector &frames, + Vector &result, idx_t rid) { + auto rdata = FlatVector::GetData(result); auto &rmask = FlatVector::Validity(result); + auto &prevs = state.prevs; + // TODO: Hack around PerfectAggregateHashTable memory leak + if (prevs.empty()) { + prevs.resize(1); + } - ModeIncluded included(fmask, dmask, bias); + ModeIncluded included(fmask, dmask); if (!state.frequency_map) { state.frequency_map = new typename STATE::Counts; } const double tau = .25; - if (state.nonzero <= tau * state.frequency_map->size() || prev.end <= frame.start || frame.end <= prev.start) { + if (state.nonzero <= tau * state.frequency_map->size()) { state.Reset(); // for f ∈ F do - for (auto f = frame.start; f < frame.end; ++f) { - if (included(f)) { - state.ModeAdd(KEY_TYPE(data[f]), f); + for (const auto &frame : frames) { + for (auto i = frame.start; i < frame.end; ++i) { + if (included(i)) { + state.ModeAdd(KEY_TYPE(data[i]), i); + } } } } else { - // for f ∈ P \ F do - for (auto p = prev.start; p < frame.start; ++p) { - if (included(p)) { - state.ModeRm(KEY_TYPE(data[p]), p); - } - } - for (auto p = frame.end; p < prev.end; ++p) { - if (included(p)) { - state.ModeRm(KEY_TYPE(data[p]), p); - } - } - - // for f ∈ F \ P do - for (auto f = frame.start; f < prev.start; ++f) { - if (included(f)) { - state.ModeAdd(KEY_TYPE(data[f]), f); - } - } - for (auto f = prev.end; f < frame.end; ++f) { - if (included(f)) { - state.ModeAdd(KEY_TYPE(data[f]), f); - } - } + using Updater = UpdateWindowState; + Updater updater(state, data, included); + AggregateExecutor::IntersectFrames(prevs, frames, updater); } if (!state.valid) { @@ -269,6 +285,8 @@ struct ModeFunction { } else { rmask.Set(rid, false); } + + prevs = frames; } static bool IgnoreNull() { @@ -277,7 +295,7 @@ struct ModeFunction { template static void Destroy(STATE &state, AggregateInputData &aggr_input_data) { - state.Destroy(); + state.~STATE(); } }; diff --git a/src/duckdb/src/core_functions/aggregate/holistic/quantile.cpp b/src/duckdb/src/core_functions/aggregate/holistic/quantile.cpp index 09f9204d8..f34087325 100644 --- a/src/duckdb/src/core_functions/aggregate/holistic/quantile.cpp +++ b/src/duckdb/src/core_functions/aggregate/holistic/quantile.cpp @@ -1,6 +1,5 @@ #include "duckdb/execution/expression_executor.hpp" #include "duckdb/core_functions/aggregate/holistic_functions.hpp" -#include "duckdb/core_functions/aggregate/quantile_enum.hpp" #include "duckdb/planner/expression.hpp" #include "duckdb/common/operator/cast_operators.hpp" #include "duckdb/common/operator/abs.hpp" @@ -37,6 +36,33 @@ inline interval_t operator-(const interval_t &lhs, const interval_t &rhs) { return Interval::FromMicro(Interval::GetMicro(lhs) - Interval::GetMicro(rhs)); } +struct FrameSet { + using Frames = vector; + + inline explicit FrameSet(const Frames &frames_p) : frames(frames_p) { + } + + inline idx_t Size() const { + idx_t result = 0; + for (const auto &frame : frames) { + result += frame.end - frame.start; + } + + return result; + } + + inline bool Contains(idx_t i) const { + for (idx_t f = 0; f < frames.size(); ++f) { + const auto &frame = frames[f]; + if (frame.start <= i && i < frame.end) { + return true; + } + } + return false; + } + const Frames &frames; +}; + template struct QuantileState { using SaveType = SAVE_TYPE; @@ -44,34 +70,41 @@ struct QuantileState { // Regular aggregation vector v; + // Windowing state + vector prevs; + // Windowed Quantile indirection vector w; - idx_t pos; + idx_t count; // Windowed MAD indirection vector m; - QuantileState() : pos(0) { + QuantileState() : count(0) { } ~QuantileState() { } - inline void SetPos(size_t pos_p) { - pos = pos_p; - if (pos >= w.size()) { - w.resize(pos); + inline void SetCount(const vector &frames) { + // TODO: Hack around PerfectAggregateHashTable memory leak + if (prevs.empty()) { + prevs.resize(1); + } + count = FrameSet(frames).Size(); + if (count >= w.size()) { + w.resize(count); } } }; struct QuantileIncluded { - inline explicit QuantileIncluded(const ValidityMask &fmask_p, const ValidityMask &dmask_p, idx_t bias_p) - : fmask(fmask_p), dmask(dmask_p), bias(bias_p) { + inline explicit QuantileIncluded(const ValidityMask &fmask_p, const ValidityMask &dmask_p) + : fmask(fmask_p), dmask(dmask_p) { } inline bool operator()(const idx_t &idx) const { - return fmask.RowIsValid(idx) && dmask.RowIsValid(idx - bias); + return fmask.RowIsValid(idx) && dmask.RowIsValid(idx); } inline bool AllValid() const { @@ -80,14 +113,40 @@ struct QuantileIncluded { const ValidityMask &fmask; const ValidityMask &dmask; - const idx_t bias; }; -void ReuseIndexes(idx_t *index, const FrameBounds &frame, const FrameBounds &prev) { - idx_t j = 0; +struct QuantileReuseUpdater { + idx_t *index; + idx_t j; - // Copy overlapping indices - for (idx_t p = 0; p < (prev.end - prev.start); ++p) { + inline QuantileReuseUpdater(idx_t *index, idx_t j) : index(index), j(j) { + } + + inline void Neither(idx_t begin, idx_t end) { + } + + inline void Left(idx_t begin, idx_t end) { + } + + inline void Right(idx_t begin, idx_t end) { + for (; begin < end; ++begin) { + index[j++] = begin; + } + } + + inline void Both(idx_t begin, idx_t end) { + } +}; + +void ReuseIndexes(idx_t *index, const vector &currs, const vector &prevs) { + + // Copy overlapping indices by scanning the previous set and copying down into holes. + // We copy instead of leaving gaps in case there are fewer values in the current frame. + FrameSet prev_set(prevs); + FrameSet curr_set(currs); + const auto prev_count = prev_set.Size(); + idx_t j = 0; + for (idx_t p = 0; p < prev_count; ++p) { auto idx = index[p]; // Shift down into any hole @@ -96,24 +155,21 @@ void ReuseIndexes(idx_t *index, const FrameBounds &frame, const FrameBounds &pre } // Skip overlapping values - if (frame.start <= idx && idx < frame.end) { + if (curr_set.Contains(idx)) { ++j; } } // Insert new indices if (j > 0) { - // Overlap: append the new ends - for (auto f = frame.start; f < prev.start; ++f, ++j) { - index[j] = f; - } - for (auto f = prev.end; f < frame.end; ++f, ++j) { - index[j] = f; - } + QuantileReuseUpdater updater(index, j); + AggregateExecutor::IntersectFrames(prevs, currs, updater); } else { // No overlap: overwrite with new values - for (auto f = frame.start; f < frame.end; ++f, ++j) { - index[j] = f; + for (const auto &curr : currs) { + for (auto idx = curr.start; idx < curr.end; ++idx) { + index[j++] = idx; + } } } } @@ -443,8 +499,6 @@ inline Value QuantileAbs(const Value &v) { } } -void BindQuantileInner(AggregateFunction &function, const LogicalType &type, QuantileSerializationType quantile_type); - struct QuantileBindData : public FunctionData { QuantileBindData() { } @@ -510,59 +564,15 @@ struct QuantileBindData : public FunctionData { deserializer.ReadProperty(100, "quantiles", raw); deserializer.ReadProperty(101, "order", result->order); deserializer.ReadProperty(102, "desc", result->desc); - QuantileSerializationType deserialization_type; - deserializer.ReadPropertyWithDefault(103, "quantile_type", deserialization_type, - QuantileSerializationType::NON_DECIMAL); - - if (deserialization_type != QuantileSerializationType::NON_DECIMAL) { - LogicalType arg_type; - deserializer.ReadProperty(104, "logical_type", arg_type); - - BindQuantileInner(function, arg_type, deserialization_type); - } - for (const auto &r : raw) { result->quantiles.emplace_back(QuantileValue(r)); } return std::move(result); } - static void SerializeDecimalDiscrete(Serializer &serializer, const optional_ptr bind_data_p, - const AggregateFunction &function) { - Serialize(serializer, bind_data_p, function); - - serializer.WritePropertyWithDefault( - 103, "quantile_type", QuantileSerializationType::DECIMAL_DISCRETE, QuantileSerializationType::NON_DECIMAL); - serializer.WriteProperty(104, "logical_type", function.arguments[0]); - } - static void SerializeDecimalDiscreteList(Serializer &serializer, const optional_ptr bind_data_p, - const AggregateFunction &function) { - - Serialize(serializer, bind_data_p, function); - - serializer.WritePropertyWithDefault(103, "quantile_type", - QuantileSerializationType::DECIMAL_DISCRETE_LIST, - QuantileSerializationType::NON_DECIMAL); - serializer.WriteProperty(104, "logical_type", function.arguments[0]); - } - static void SerializeDecimalContinuous(Serializer &serializer, const optional_ptr bind_data_p, - const AggregateFunction &function) { - Serialize(serializer, bind_data_p, function); - - serializer.WritePropertyWithDefault(103, "quantile_type", - QuantileSerializationType::DECIMAL_CONTINUOUS, - QuantileSerializationType::NON_DECIMAL); - serializer.WriteProperty(104, "logical_type", function.arguments[0]); - } - static void SerializeDecimalContinuousList(Serializer &serializer, const optional_ptr bind_data_p, - const AggregateFunction &function) { - - Serialize(serializer, bind_data_p, function); - - serializer.WritePropertyWithDefault( - 103, "quantile_type", QuantileSerializationType::DECIMAL_CONTINUOUS_LIST, - QuantileSerializationType::NON_DECIMAL); - serializer.WriteProperty(104, "logical_type", function.arguments[0]); + static void SerializeDecimal(Serializer &serializer, const optional_ptr bind_data_p, + const AggregateFunction &function) { + throw NotImplementedException("FIXME: serializing quantiles with decimals is not supported right now"); } vector quantiles; @@ -635,16 +645,17 @@ struct QuantileScalarOperation : public QuantileOperation { template static void Window(const INPUT_TYPE *data, const ValidityMask &fmask, const ValidityMask &dmask, - AggregateInputData &aggr_input_data, STATE &state, const FrameBounds &frame, - const FrameBounds &prev, Vector &result, idx_t ridx, idx_t bias) { + AggregateInputData &aggr_input_data, STATE &state, const vector &frames, + Vector &result, idx_t ridx) { auto rdata = FlatVector::GetData(result); auto &rmask = FlatVector::Validity(result); - QuantileIncluded included(fmask, dmask, bias); + QuantileIncluded included(fmask, dmask); // Lazily initialise frame state - auto prev_pos = state.pos; - state.SetPos(frame.end - frame.start); + const auto prev_count = state.count; + auto &prevs = state.prevs; + state.SetCount(frames); auto index = state.w.data(); D_ASSERT(index); @@ -656,27 +667,27 @@ struct QuantileScalarOperation : public QuantileOperation { const auto &q = bind_data.quantiles[0]; bool replace = false; - if (frame.start == prev.start + 1 && frame.end == prev.end + 1) { + if (frames.size() == 1 && frames[0].start == prevs[0].start + 1 && frames[0].end == prevs[0].end + 1) { // Fixed frame size - const auto j = ReplaceIndex(index, frame, prev); + const auto j = ReplaceIndex(index, frames[0], prevs[0]); // We can only replace if the number of NULLs has not changed - if (included.AllValid() || included(prev.start) == included(prev.end)) { - Interpolator interp(q, prev_pos, false); + if (included.AllValid() || included(prevs[0].start) == included(prevs[0].end)) { + Interpolator interp(q, prev_count, false); replace = CanReplace(index, data, j, interp.FRN, interp.CRN, included); if (replace) { - state.pos = prev_pos; + state.count = prev_count; } } } else { - ReuseIndexes(index, frame, prev); + ReuseIndexes(index, frames, prevs); } if (!replace && !included.AllValid()) { // Remove the NULLs - state.pos = std::partition(index, index + state.pos, included) - index; + state.count = std::partition(index, index + state.count, included) - index; } - if (state.pos) { - Interpolator interp(q, state.pos, false); + if (state.count) { + Interpolator interp(q, state.count, false); using ID = QuantileIndirect; ID indirect(data); @@ -685,6 +696,8 @@ struct QuantileScalarOperation : public QuantileOperation { } else { rmask.Set(ridx, false); } + + prevs = frames; } }; @@ -783,12 +796,12 @@ struct QuantileListOperation : public QuantileOperation { template static void Window(const INPUT_TYPE *data, const ValidityMask &fmask, const ValidityMask &dmask, - AggregateInputData &aggr_input_data, STATE &state, const FrameBounds &frame, - const FrameBounds &prev, Vector &list, idx_t lidx, idx_t bias) { + AggregateInputData &aggr_input_data, STATE &state, const vector &frames, + Vector &list, idx_t lidx) { D_ASSERT(aggr_input_data.bind_data); auto &bind_data = aggr_input_data.bind_data->Cast(); - QuantileIncluded included(fmask, dmask, bias); + QuantileIncluded included(fmask, dmask); // Result is a constant LIST with a fixed length auto ldata = FlatVector::GetData(list); @@ -803,8 +816,9 @@ struct QuantileListOperation : public QuantileOperation { auto rdata = FlatVector::GetData(result); // Lazily initialise frame state - auto prev_pos = state.pos; - state.SetPos(frame.end - frame.start); + const auto prev_count = state.count; + auto &prevs = state.prevs; + state.SetCount(frames); auto index = state.w.data(); @@ -814,20 +828,20 @@ struct QuantileListOperation : public QuantileOperation { // So if a replaced index in an IQR is located between Q25 and Q50, but has a value below Q25, // then Q25 must be recomputed, but Q50 and Q75 are unaffected. // For a single element list, this reduces to the scalar case. - std::pair replaceable {state.pos, 0}; - if (frame.start == prev.start + 1 && frame.end == prev.end + 1) { + std::pair replaceable {state.count, 0}; + if (frames.size() == 1 && frames[0].start == prevs[0].start + 1 && frames[0].end == prevs[0].end + 1) { // Fixed frame size - const auto j = ReplaceIndex(index, frame, prev); + const auto j = ReplaceIndex(index, frames[0], prevs[0]); // We can only replace if the number of NULLs has not changed - if (included.AllValid() || included(prev.start) == included(prev.end)) { + if (included.AllValid() || included(prevs[0].start) == included(prevs[0].end)) { for (const auto &q : bind_data.order) { const auto &quantile = bind_data.quantiles[q]; - Interpolator interp(quantile, prev_pos, false); + Interpolator interp(quantile, prev_count, false); const auto replace = CanReplace(index, data, j, interp.FRN, interp.CRN, included); if (replace < 0) { // Replacement is before this quantile, so the rest will be replaceable too. replaceable.first = MinValue(replaceable.first, interp.FRN); - replaceable.second = prev_pos; + replaceable.second = prev_count; break; } else if (replace > 0) { // Replacement is after this quantile, so everything before it is replaceable too. @@ -836,24 +850,24 @@ struct QuantileListOperation : public QuantileOperation { } } if (replaceable.first < replaceable.second) { - state.pos = prev_pos; + state.count = prev_count; } } } else { - ReuseIndexes(index, frame, prev); + ReuseIndexes(index, frames, prevs); } if (replaceable.first >= replaceable.second && !included.AllValid()) { // Remove the NULLs - state.pos = std::partition(index, index + state.pos, included) - index; + state.count = std::partition(index, index + state.count, included) - index; } - if (state.pos) { + if (state.count) { using ID = QuantileIndirect; ID indirect(data); for (const auto &q : bind_data.order) { const auto &quantile = bind_data.quantiles[q]; - Interpolator interp(quantile, state.pos, false); + Interpolator interp(quantile, state.count, false); if (replaceable.first <= interp.FRN && interp.CRN <= replaceable.second) { rdata[lentry.offset + q] = interp.template Replace(index, result, indirect); } else { @@ -873,6 +887,8 @@ struct QuantileListOperation : public QuantileOperation { } else { lmask.Set(lidx, false); } + + prevs = frames; } }; @@ -1141,23 +1157,24 @@ struct MedianAbsoluteDeviationOperation : public QuantileOperation { template static void Window(const INPUT_TYPE *data, const ValidityMask &fmask, const ValidityMask &dmask, - AggregateInputData &aggr_input_data, STATE &state, const FrameBounds &frame, - const FrameBounds &prev, Vector &result, idx_t ridx, idx_t bias) { + AggregateInputData &aggr_input_data, STATE &state, const vector &frames, + Vector &result, idx_t ridx) { auto rdata = FlatVector::GetData(result); auto &rmask = FlatVector::Validity(result); - QuantileIncluded included(fmask, dmask, bias); + QuantileIncluded included(fmask, dmask); // Lazily initialise frame state - auto prev_pos = state.pos; - state.SetPos(frame.end - frame.start); + auto prev_count = state.count; + auto &prevs = state.prevs; + state.SetCount(frames); auto index = state.w.data(); D_ASSERT(index); // We need a second index for the second pass. - if (state.pos > state.m.size()) { - state.m.resize(state.pos); + if (state.count > state.m.size()) { + state.m.resize(state.count); } auto index2 = state.m.data(); @@ -1166,8 +1183,8 @@ struct MedianAbsoluteDeviationOperation : public QuantileOperation { // The replacement trick does not work on the second index because if // the median has changed, the previous order is not correct. // It is probably close, however, and so reuse is helpful. - ReuseIndexes(index2, frame, prev); - std::partition(index2, index2 + state.pos, included); + ReuseIndexes(index2, frames, prevs); + std::partition(index2, index2 + state.count, included); // Find the two positions needed for the median D_ASSERT(aggr_input_data.bind_data); @@ -1176,28 +1193,28 @@ struct MedianAbsoluteDeviationOperation : public QuantileOperation { const auto &q = bind_data.quantiles[0]; bool replace = false; - if (frame.start == prev.start + 1 && frame.end == prev.end + 1) { + if (frames.size() == 1 && frames[0].start == prevs[0].start + 1 && frames[0].end == prevs[0].end + 1) { // Fixed frame size - const auto j = ReplaceIndex(index, frame, prev); + const auto j = ReplaceIndex(index, frames[0], prevs[0]); // We can only replace if the number of NULLs has not changed - if (included.AllValid() || included(prev.start) == included(prev.end)) { - Interpolator interp(q, prev_pos, false); + if (included.AllValid() || included(prevs[0].start) == included(prevs[0].end)) { + Interpolator interp(q, prev_count, false); replace = CanReplace(index, data, j, interp.FRN, interp.CRN, included); if (replace) { - state.pos = prev_pos; + state.count = prev_count; } } } else { - ReuseIndexes(index, frame, prev); + ReuseIndexes(index, frames, prevs); } if (!replace && !included.AllValid()) { // Remove the NULLs - state.pos = std::partition(index, index + state.pos, included) - index; + state.count = std::partition(index, index + state.count, included) - index; } - if (state.pos) { - Interpolator interp(q, state.pos, false); + if (state.count) { + Interpolator interp(q, state.count, false); // Compute or replace median from the first index using ID = QuantileIndirect; @@ -1215,6 +1232,8 @@ struct MedianAbsoluteDeviationOperation : public QuantileOperation { } else { rmask.Set(ridx, false); } + + prevs = frames; } }; @@ -1279,7 +1298,7 @@ unique_ptr BindMedianDecimal(ClientContext &context, AggregateFunc function = GetDiscreteQuantileAggregateFunction(arguments[0]->return_type); function.name = "median"; - function.serialize = QuantileBindData::SerializeDecimalDiscrete; + function.serialize = QuantileBindData::SerializeDecimal; function.deserialize = QuantileBindData::Deserialize; function.order_dependent = AggregateOrderDependent::NOT_ORDER_DEPENDENT; return bind_data; @@ -1330,62 +1349,50 @@ unique_ptr BindQuantile(ClientContext &context, AggregateFunction return make_uniq(quantiles); } -void BindQuantileInner(AggregateFunction &function, const LogicalType &type, QuantileSerializationType quantile_type) { - switch (quantile_type) { - case QuantileSerializationType::DECIMAL_DISCRETE: - function = GetDiscreteQuantileAggregateFunction(type); - function.serialize = QuantileBindData::SerializeDecimalDiscrete; - function.name = "quantile_disc"; - break; - case QuantileSerializationType::DECIMAL_DISCRETE_LIST: - function = GetDiscreteQuantileListAggregateFunction(type); - function.serialize = QuantileBindData::SerializeDecimalDiscreteList; - function.name = "quantile_disc"; - break; - case QuantileSerializationType::DECIMAL_CONTINUOUS: - function = GetContinuousQuantileAggregateFunction(type); - function.serialize = QuantileBindData::SerializeDecimalContinuous; - function.name = "quantile_cont"; - break; - case QuantileSerializationType::DECIMAL_CONTINUOUS_LIST: - function = GetContinuousQuantileListAggregateFunction(type); - function.serialize = QuantileBindData::SerializeDecimalContinuousList; - function.name = "quantile_cont"; - break; - case QuantileSerializationType::NON_DECIMAL: - throw SerializationException("NON_DECIMAL is not a valid quantile_type for BindQuantileInner"); - } - function.deserialize = QuantileBindData::Deserialize; - function.order_dependent = AggregateOrderDependent::NOT_ORDER_DEPENDENT; -} - unique_ptr BindDiscreteQuantileDecimal(ClientContext &context, AggregateFunction &function, vector> &arguments) { auto bind_data = BindQuantile(context, function, arguments); - BindQuantileInner(function, arguments[0]->return_type, QuantileSerializationType::DECIMAL_DISCRETE); + function = GetDiscreteQuantileAggregateFunction(arguments[0]->return_type); + function.name = "quantile_disc"; + function.serialize = QuantileBindData::SerializeDecimal; + function.deserialize = QuantileBindData::Deserialize; + function.order_dependent = AggregateOrderDependent::NOT_ORDER_DEPENDENT; return bind_data; } unique_ptr BindDiscreteQuantileDecimalList(ClientContext &context, AggregateFunction &function, vector> &arguments) { auto bind_data = BindQuantile(context, function, arguments); - BindQuantileInner(function, arguments[0]->return_type, QuantileSerializationType::DECIMAL_DISCRETE_LIST); + function = GetDiscreteQuantileListAggregateFunction(arguments[0]->return_type); + function.name = "quantile_disc"; + function.serialize = QuantileBindData::SerializeDecimal; + function.deserialize = QuantileBindData::Deserialize; + function.order_dependent = AggregateOrderDependent::NOT_ORDER_DEPENDENT; return bind_data; } unique_ptr BindContinuousQuantileDecimal(ClientContext &context, AggregateFunction &function, vector> &arguments) { auto bind_data = BindQuantile(context, function, arguments); - BindQuantileInner(function, arguments[0]->return_type, QuantileSerializationType::DECIMAL_CONTINUOUS); + function = GetContinuousQuantileAggregateFunction(arguments[0]->return_type); + function.name = "quantile_cont"; + function.serialize = QuantileBindData::SerializeDecimal; + function.deserialize = QuantileBindData::Deserialize; + function.order_dependent = AggregateOrderDependent::NOT_ORDER_DEPENDENT; return bind_data; } unique_ptr BindContinuousQuantileDecimalList(ClientContext &context, AggregateFunction &function, vector> &arguments) { auto bind_data = BindQuantile(context, function, arguments); - BindQuantileInner(function, arguments[0]->return_type, QuantileSerializationType::DECIMAL_CONTINUOUS_LIST); + function = GetContinuousQuantileListAggregateFunction(arguments[0]->return_type); + function.name = "quantile_cont"; + function.serialize = QuantileBindData::SerializeDecimal; + function.deserialize = QuantileBindData::Deserialize; + function.order_dependent = AggregateOrderDependent::NOT_ORDER_DEPENDENT; return bind_data; } + static bool CanInterpolate(const LogicalType &type) { switch (type.id()) { case LogicalTypeId::INTERVAL: diff --git a/src/duckdb/src/core_functions/aggregate/nested/list.cpp b/src/duckdb/src/core_functions/aggregate/nested/list.cpp index a7d4d7428..20f167a75 100644 --- a/src/duckdb/src/core_functions/aggregate/nested/list.cpp +++ b/src/duckdb/src/core_functions/aggregate/nested/list.cpp @@ -148,8 +148,8 @@ static void ListFinalize(Vector &states_vector, AggregateInputData &aggr_input_d } static void ListWindow(Vector inputs[], const ValidityMask &filter_mask, AggregateInputData &aggr_input_data, - idx_t input_count, data_ptr_t state, const FrameBounds &frame, const FrameBounds &prev, - Vector &result, idx_t rid, idx_t bias) { + idx_t input_count, data_ptr_t state, const vector &frames, Vector &result, + idx_t rid) { auto &list_bind_data = aggr_input_data.bind_data->Cast(); LinkedList linked_list; @@ -160,11 +160,15 @@ static void ListWindow(Vector inputs[], const ValidityMask &filter_mask, Aggrega auto &input = inputs[0]; // FIXME: we unify more values than necessary (count is frame.end) + const auto count = frames.back().end; + RecursiveUnifiedVectorFormat input_data; - Vector::RecursiveToUnifiedFormat(input, frame.end, input_data); + Vector::RecursiveToUnifiedFormat(input, count, input_data); - for (idx_t i = frame.start; i < frame.end; i++) { - list_bind_data.functions.AppendRow(aggr_input_data.allocator, linked_list, input_data, i); + for (const auto &frame : frames) { + for (idx_t i = frame.start; i < frame.end; i++) { + list_bind_data.functions.AppendRow(aggr_input_data.allocator, linked_list, input_data, i); + } } // FINALIZE step diff --git a/src/duckdb/src/core_functions/function_list.cpp b/src/duckdb/src/core_functions/function_list.cpp index 299fd59e2..83cff4563 100644 --- a/src/duckdb/src/core_functions/function_list.cpp +++ b/src/duckdb/src/core_functions/function_list.cpp @@ -17,6 +17,7 @@ #include "duckdb/core_functions/scalar/string_functions.hpp" #include "duckdb/core_functions/scalar/struct_functions.hpp" #include "duckdb/core_functions/scalar/union_functions.hpp" +#include "duckdb/core_functions/scalar/array_functions.hpp" #include "duckdb/core_functions/scalar/debug_functions.hpp" namespace duckdb { @@ -73,13 +74,20 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayAggrFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayAggregateFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayApplyFun), + DUCKDB_SCALAR_FUNCTION_SET(ArrayCosineSimilarityFun), + DUCKDB_SCALAR_FUNCTION_SET(ArrayCrossProductFun), + DUCKDB_SCALAR_FUNCTION_SET(ArrayDistanceFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayDistinctFun), + DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArrayDotProductFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayFilterFun), + DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArrayGradeUpFun), + DUCKDB_SCALAR_FUNCTION_SET(ArrayInnerProductFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArrayReverseSortFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArraySliceFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArraySortFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayTransformFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayUniqueFun), + DUCKDB_SCALAR_FUNCTION(ArrayValueFun), DUCKDB_SCALAR_FUNCTION(ASCIIFun), DUCKDB_SCALAR_FUNCTION(AsinFun), DUCKDB_SCALAR_FUNCTION(AtanFun), @@ -167,6 +175,7 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION(GetBitFun), DUCKDB_SCALAR_FUNCTION(CurrentTimeFun), DUCKDB_SCALAR_FUNCTION(GetCurrentTimestampFun), + DUCKDB_SCALAR_FUNCTION_SET_ALIAS(GradeUpFun), DUCKDB_SCALAR_FUNCTION_SET(GreatestFun), DUCKDB_SCALAR_FUNCTION_SET(GreatestCommonDivisorFun), DUCKDB_AGGREGATE_FUNCTION_SET_ALIAS(GroupConcatFun), @@ -205,6 +214,7 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION(ListDistinctFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ListDotProductFun), DUCKDB_SCALAR_FUNCTION(ListFilterFun), + DUCKDB_SCALAR_FUNCTION_SET(ListGradeUpFun), DUCKDB_SCALAR_FUNCTION_SET(ListInnerProductFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ListPackFun), DUCKDB_SCALAR_FUNCTION_SET(ListReverseSortFun), @@ -213,9 +223,8 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION(ListTransformFun), DUCKDB_SCALAR_FUNCTION(ListUniqueFun), DUCKDB_SCALAR_FUNCTION(ListValueFun), - DUCKDB_AGGREGATE_FUNCTION_SET_ALIAS(ListaggFun), DUCKDB_SCALAR_FUNCTION(LnFun), - DUCKDB_SCALAR_FUNCTION_SET(LogFun), + DUCKDB_SCALAR_FUNCTION_ALIAS(LogFun), DUCKDB_SCALAR_FUNCTION(Log10Fun), DUCKDB_SCALAR_FUNCTION(Log2Fun), DUCKDB_SCALAR_FUNCTION(LpadFun), diff --git a/src/duckdb/src/core_functions/lambda_functions.cpp b/src/duckdb/src/core_functions/lambda_functions.cpp new file mode 100644 index 000000000..374e254ea --- /dev/null +++ b/src/duckdb/src/core_functions/lambda_functions.cpp @@ -0,0 +1,423 @@ +#include "duckdb/core_functions/lambda_functions.hpp" + +#include "duckdb/common/serializer/serializer.hpp" +#include "duckdb/common/serializer/deserializer.hpp" + +#include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" + +namespace duckdb { + +//===--------------------------------------------------------------------===// +// Helper functions +//===--------------------------------------------------------------------===// + +//! LambdaColumnInfo holds information for preparing the input vectors. We prepare the input vectors +//! for executing a lambda expression on STANDARD_VECTOR_SIZE list child elements at a time. +struct LambdaColumnInfo { + explicit LambdaColumnInfo(Vector &vector) : vector(vector), sel(SelectionVector(STANDARD_VECTOR_SIZE)) {}; + + //! The original vector taken from args + reference vector; + //! The selection vector to slice the original vector + SelectionVector sel; + //! The unified vector format of the original vector + UnifiedVectorFormat format; +}; + +//! LambdaExecuteInfo holds information for executing the lambda expression on an input chunk and +//! a resulting lambda chunk. +struct LambdaExecuteInfo { + LambdaExecuteInfo(ClientContext &context, const Expression &lambda_expr, const DataChunk &args, + const bool has_index, const Vector &child_vector) + : has_index(has_index) { + + expr_executor = make_uniq(context, lambda_expr); + + // get the input types for the input chunk + vector input_types; + if (has_index) { + input_types.push_back(LogicalType::BIGINT); + } + input_types.push_back(child_vector.GetType()); + for (idx_t i = 1; i < args.ColumnCount(); i++) { + input_types.push_back(args.data[i].GetType()); + } + + // get the result types + vector result_types {lambda_expr.return_type}; + + // initialize the data chunks + input_chunk.InitializeEmpty(input_types); + lambda_chunk.Initialize(Allocator::DefaultAllocator(), result_types); + }; + + //! The expression executor that executes the lambda expression + unique_ptr expr_executor; + //! The input chunk on which we execute the lambda expression + DataChunk input_chunk; + //! The chunk holding the result of executing the lambda expression + DataChunk lambda_chunk; + //! True, if this lambda expression expects an index vector in the input chunk + bool has_index; +}; + +//! A helper struct with information that is specific to the list_filter function +struct ListFilterInfo { + //! The new list lengths after filtering out elements + vector entry_lengths; + //! The length of the current list + idx_t length = 0; + //! The offset of the current list + idx_t offset = 0; + //! The current row index + idx_t row_idx = 0; + //! The length of the source list + idx_t src_length = 0; +}; + +//! ListTransformFunctor contains list_transform specific functionality +struct ListTransformFunctor { + static void ReserveNewLengths(vector &, const idx_t) { + // NOP + } + static void PushEmptyList(vector &) { + // NOP + } + //! Sets the list entries of the result vector + static void SetResultEntry(list_entry_t *result_entries, idx_t &offset, const list_entry_t &entry, + const idx_t row_idx, vector &) { + result_entries[row_idx].offset = offset; + result_entries[row_idx].length = entry.length; + offset += entry.length; + } + //! Appends the lambda vector to the result's child vector + static void AppendResult(Vector &result, Vector &lambda_vector, const idx_t elem_cnt, list_entry_t *, + ListFilterInfo &, LambdaExecuteInfo &) { + ListVector::Append(result, lambda_vector, elem_cnt, 0); + } +}; + +//! ListFilterFunctor contains list_filter specific functionality +struct ListFilterFunctor { + //! Initializes the entry_lengths vector + static void ReserveNewLengths(vector &entry_lengths, const idx_t row_count) { + entry_lengths.reserve(row_count); + } + //! Pushes an empty list to the entry_lengths vector + static void PushEmptyList(vector &entry_lengths) { + entry_lengths.emplace_back(0); + } + //! Pushes the length of the original list to the entry_lengths vector + static void SetResultEntry(list_entry_t *, idx_t &, const list_entry_t &entry, const idx_t, + vector &entry_lengths) { + entry_lengths.push_back(entry.length); + } + //! Uses the lambda vector to filter the incoming list and to append the filtered list to the result vector + static void AppendResult(Vector &result, Vector &lambda_vector, const idx_t elem_cnt, list_entry_t *result_entries, + ListFilterInfo &info, LambdaExecuteInfo &execute_info) { + + idx_t count = 0; + SelectionVector sel(elem_cnt); + UnifiedVectorFormat lambda_data; + lambda_vector.ToUnifiedFormat(elem_cnt, lambda_data); + + auto lambda_values = UnifiedVectorFormat::GetData(lambda_data); + auto &lambda_validity = lambda_data.validity; + + // compute the new lengths and offsets, and create a selection vector + for (idx_t i = 0; i < elem_cnt; i++) { + auto entry_idx = lambda_data.sel->get_index(i); + + // set length and offset of empty lists + while (info.row_idx < info.entry_lengths.size() && !info.entry_lengths[info.row_idx]) { + result_entries[info.row_idx].offset = info.offset; + result_entries[info.row_idx].length = 0; + info.row_idx++; + } + + // found a true value + if (lambda_validity.RowIsValid(entry_idx) && lambda_values[entry_idx]) { + sel.set_index(count++, i); + info.length++; + } + + info.src_length++; + + // we traversed the entire source list + if (info.entry_lengths[info.row_idx] == info.src_length) { + // set the offset and length of the result entry + result_entries[info.row_idx].offset = info.offset; + result_entries[info.row_idx].length = info.length; + + // reset all other fields + info.offset += info.length; + info.row_idx++; + info.length = 0; + info.src_length = 0; + } + } + + // set length and offset of all remaining empty lists + while (info.row_idx < info.entry_lengths.size() && !info.entry_lengths[info.row_idx]) { + result_entries[info.row_idx].offset = info.offset; + result_entries[info.row_idx].length = 0; + info.row_idx++; + } + + // slice the input chunk's corresponding vector to get the new lists + // and append them to the result + auto source_list_idx = execute_info.has_index ? 1 : 0; + Vector result_lists(execute_info.input_chunk.data[source_list_idx], sel, count); + ListVector::Append(result, result_lists, count, 0); + } +}; + +vector GetColumnInfo(DataChunk &args, const idx_t row_count) { + + vector data; + // skip the input list and then insert all remaining input vectors + for (idx_t i = 1; i < args.ColumnCount(); i++) { + data.emplace_back(args.data[i]); + args.data[i].ToUnifiedFormat(row_count, data.back().format); + } + return data; +} + +vector> GetInconstantColumnInfo(vector &data) { + + vector> inconstant_info; + for (auto &entry : data) { + if (entry.vector.get().GetVectorType() != VectorType::CONSTANT_VECTOR) { + inconstant_info.push_back(entry); + } + } + return inconstant_info; +} + +void ExecuteExpression(const idx_t elem_cnt, const LambdaColumnInfo &column_info, + const vector &column_infos, const Vector &index_vector, + LambdaExecuteInfo &info) { + + info.input_chunk.SetCardinality(elem_cnt); + info.lambda_chunk.SetCardinality(elem_cnt); + + // slice the child vector + Vector slice(column_info.vector, column_info.sel, elem_cnt); + + // reference the child vector (and the index vector) + if (info.has_index) { + info.input_chunk.data[0].Reference(index_vector); + info.input_chunk.data[1].Reference(slice); + } else { + info.input_chunk.data[0].Reference(slice); + } + idx_t slice_offset = info.has_index ? 2 : 1; + + // (slice and) reference the other columns + vector slices; + for (idx_t i = 0; i < column_infos.size(); i++) { + + if (column_infos[i].vector.get().GetVectorType() == VectorType::CONSTANT_VECTOR) { + // only reference constant vectors + info.input_chunk.data[i + slice_offset].Reference(column_infos[i].vector); + + } else { + // slice inconstant vectors + slices.emplace_back(column_infos[i].vector, column_infos[i].sel, elem_cnt); + info.input_chunk.data[i + slice_offset].Reference(slices.back()); + } + } + + // execute the lambda expression + info.expr_executor->Execute(info.input_chunk, info.lambda_chunk); +} + +//===--------------------------------------------------------------------===// +// ListLambdaBindData +//===--------------------------------------------------------------------===// + +unique_ptr ListLambdaBindData::Copy() const { + auto lambda_expr_copy = lambda_expr ? lambda_expr->Copy() : nullptr; + return make_uniq(return_type, std::move(lambda_expr_copy), has_index); +} + +bool ListLambdaBindData::Equals(const FunctionData &other_p) const { + auto &other = other_p.Cast(); + return Expression::Equals(lambda_expr, other.lambda_expr) && return_type == other.return_type && + has_index == other.has_index; +} + +void ListLambdaBindData::Serialize(Serializer &serializer, const optional_ptr bind_data_p, + const ScalarFunction &) { + auto &bind_data = bind_data_p->Cast(); + serializer.WriteProperty(100, "return_type", bind_data.return_type); + serializer.WritePropertyWithDefault(101, "lambda_expr", bind_data.lambda_expr, unique_ptr()); + serializer.WriteProperty(102, "has_index", bind_data.has_index); +} + +unique_ptr ListLambdaBindData::Deserialize(Deserializer &deserializer, ScalarFunction &) { + auto return_type = deserializer.ReadProperty(100, "return_type"); + auto lambda_expr = + deserializer.ReadPropertyWithDefault>(101, "lambda_expr", unique_ptr()); + auto has_index = deserializer.ReadProperty(102, "has_index"); + return make_uniq(return_type, std::move(lambda_expr), has_index); +} + +//===--------------------------------------------------------------------===// +// LambdaFunctions +//===--------------------------------------------------------------------===// + +LogicalType LambdaFunctions::BindBinaryLambda(const idx_t parameter_idx, const LogicalType &list_child_type) { + switch (parameter_idx) { + case 0: + return list_child_type; + case 1: + return LogicalType::BIGINT; + default: + throw BinderException("This lambda function only supports up to two lambda parameters!"); + } +} + +template +void ExecuteLambda(DataChunk &args, ExpressionState &state, Vector &result) { + + auto row_count = args.size(); + Vector &list_column = args.data[0]; + + result.SetVectorType(VectorType::FLAT_VECTOR); + auto result_entries = FlatVector::GetData(result); + auto &result_validity = FlatVector::Validity(result); + + if (list_column.GetType().id() == LogicalTypeId::SQLNULL) { + result_validity.SetInvalid(0); + return; + } + + // get the lambda expression + auto &func_expr = state.expr.Cast(); + auto &bind_info = func_expr.bind_info->Cast(); + auto &lambda_expr = bind_info.lambda_expr; + bool has_side_effects = lambda_expr->HasSideEffects(); + + // get the list column entries + UnifiedVectorFormat list_column_format; + list_column.ToUnifiedFormat(row_count, list_column_format); + auto list_entries = UnifiedVectorFormat::GetData(list_column_format); + + // special-handling for the child_vector + auto child_vector_size = ListVector::GetListSize(list_column); + auto &child_vector = ListVector::GetEntry(list_column); + LambdaColumnInfo child_info(child_vector); + child_vector.ToUnifiedFormat(child_vector_size, child_info.format); + + // get the lambda column data for all other input vectors + auto column_infos = GetColumnInfo(args, row_count); + auto inconstant_column_infos = GetInconstantColumnInfo(column_infos); + + // get the expression executor + LambdaExecuteInfo execute_info(state.GetContext(), *lambda_expr, args, bind_info.has_index, child_vector); + + // get list_filter specific info + ListFilterInfo list_filter_info; + FUNCTION_FUNCTOR::ReserveNewLengths(list_filter_info.entry_lengths, row_count); + + // additional index vector + Vector index_vector(LogicalType::BIGINT); + + // loop over the child entries and create chunks to be executed by the expression executor + idx_t elem_cnt = 0; + idx_t offset = 0; + for (idx_t row_idx = 0; row_idx < row_count; row_idx++) { + + auto list_idx = list_column_format.sel->get_index(row_idx); + const auto &list_entry = list_entries[list_idx]; + + // set the result to NULL for this row + if (!list_column_format.validity.RowIsValid(list_idx)) { + result_validity.SetInvalid(row_idx); + FUNCTION_FUNCTOR::PushEmptyList(list_filter_info.entry_lengths); + continue; + } + + FUNCTION_FUNCTOR::SetResultEntry(result_entries, offset, list_entry, row_idx, list_filter_info.entry_lengths); + + // empty list, nothing to execute + if (list_entry.length == 0) { + continue; + } + + // iterate the elements of the current list and create the corresponding selection vectors + for (idx_t child_idx = 0; child_idx < list_entry.length; child_idx++) { + + // reached STANDARD_VECTOR_SIZE elements + if (elem_cnt == STANDARD_VECTOR_SIZE) { + + execute_info.lambda_chunk.Reset(); + ExecuteExpression(elem_cnt, child_info, column_infos, index_vector, execute_info); + auto &lambda_vector = execute_info.lambda_chunk.data[0]; + + FUNCTION_FUNCTOR::AppendResult(result, lambda_vector, elem_cnt, result_entries, list_filter_info, + execute_info); + elem_cnt = 0; + } + + // adjust indexes for slicing + child_info.sel.set_index(elem_cnt, list_entry.offset + child_idx); + for (auto &entry : inconstant_column_infos) { + entry.get().sel.set_index(elem_cnt, row_idx); + } + + // set the index vector + if (bind_info.has_index) { + index_vector.SetValue(elem_cnt, Value::BIGINT(child_idx + 1)); + } + + elem_cnt++; + } + } + + execute_info.lambda_chunk.Reset(); + ExecuteExpression(elem_cnt, child_info, column_infos, index_vector, execute_info); + auto &lambda_vector = execute_info.lambda_chunk.data[0]; + + FUNCTION_FUNCTOR::AppendResult(result, lambda_vector, elem_cnt, result_entries, list_filter_info, execute_info); + + if (args.AllConstant() && !has_side_effects) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +unique_ptr LambdaFunctions::ListLambdaBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments, + const bool has_index) { + + // NULL list parameter + if (arguments[0]->return_type.id() == LogicalTypeId::SQLNULL) { + bound_function.arguments[0] = LogicalType::SQLNULL; + bound_function.return_type = LogicalType::SQLNULL; + return make_uniq(bound_function.return_type, nullptr); + } + // prepared statements + if (arguments[0]->return_type.id() == LogicalTypeId::UNKNOWN) { + throw ParameterNotResolvedException(); + } + + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + D_ASSERT(arguments[0]->return_type.id() == LogicalTypeId::LIST); + + // get the lambda expression and put it in the bind info + auto &bound_lambda_expr = arguments[1]->Cast(); + auto lambda_expr = std::move(bound_lambda_expr.lambda_expr); + return make_uniq(bound_function.return_type, std::move(lambda_expr), has_index); +} + +void LambdaFunctions::ListTransformFunction(DataChunk &args, ExpressionState &state, Vector &result) { + ExecuteLambda(args, state, result); +} + +void LambdaFunctions::ListFilterFunction(DataChunk &args, ExpressionState &state, Vector &result) { + ExecuteLambda(args, state, result); +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/array/array_functions.cpp b/src/duckdb/src/core_functions/scalar/array/array_functions.cpp new file mode 100644 index 000000000..ac1f7664e --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/array/array_functions.cpp @@ -0,0 +1,294 @@ +#include "duckdb/core_functions/scalar/array_functions.hpp" +#include + +namespace duckdb { + +//------------------------------------------------------------------------------ +// Functors +//------------------------------------------------------------------------------ + +struct InnerProductOp { + static constexpr const char *NAME = "array_inner_product"; + + template + inline static TYPE *GetResultData(Vector &result_vec) { + return FlatVector::GetData(result_vec); + } + + template + inline static void Operation(TYPE *l_data, idx_t l_idx, TYPE *r_data, idx_t r_idx, TYPE *result_data, + idx_t result_idx, idx_t size) { + + TYPE inner_product = 0; + + auto l_ptr = l_data + (l_idx * size); + auto r_ptr = r_data + (r_idx * size); + + for (idx_t elem_idx = 0; elem_idx < size; elem_idx++) { + auto x = *l_ptr++; + auto y = *r_ptr++; + inner_product += x * y; + } + + result_data[result_idx] = inner_product; + } +}; + +struct DistanceOp { + static constexpr const char *NAME = "array_distance"; + + template + inline static TYPE *GetResultData(Vector &result_vec) { + return FlatVector::GetData(result_vec); + } + + template + inline static void Operation(TYPE *l_data, idx_t l_idx, TYPE *r_data, idx_t r_idx, TYPE *result_data, + idx_t result_idx, idx_t size) { + + TYPE distance = 0; + + auto l_ptr = l_data + (l_idx * size); + auto r_ptr = r_data + (r_idx * size); + + for (idx_t elem_idx = 0; elem_idx < size; elem_idx++) { + auto x = *l_ptr++; + auto y = *r_ptr++; + auto diff = x - y; + distance += diff * diff; + } + + result_data[result_idx] = std::sqrt(distance); + } +}; + +struct CosineSimilarityOp { + static constexpr const char *NAME = "array_cosine_similarity"; + + template + inline static TYPE *GetResultData(Vector &result_vec) { + return FlatVector::GetData(result_vec); + } + + template + inline static void Operation(TYPE *l_data, idx_t l_idx, TYPE *r_data, idx_t r_idx, TYPE *result_data, + idx_t result_idx, idx_t size) { + + TYPE distance = 0; + TYPE norm_l = 0; + TYPE norm_r = 0; + + auto l_ptr = l_data + (l_idx * size); + auto r_ptr = r_data + (r_idx * size); + + for (idx_t i = 0; i < size; i++) { + auto x = *l_ptr++; + auto y = *r_ptr++; + distance += x * y; + norm_l += x * x; + norm_r += y * y; + } + + auto similarity = distance / (std::sqrt(norm_l) * std::sqrt(norm_r)); + + // clamp to [-1, 1] to avoid floating point errors + result_data[result_idx] = std::max(static_cast(-1), std::min(similarity, static_cast(1))); + } +}; + +struct CrossProductOp { + static constexpr const char *NAME = "array_cross_product"; + + template + inline static TYPE *GetResultData(Vector &result_vec) { + // Since we return an array here, we need to get the data pointer of the child + auto &child = ArrayVector::GetEntry(result_vec); + return FlatVector::GetData(child); + } + + template + inline static void Operation(TYPE *l_data, idx_t l_idx, TYPE *r_data, idx_t r_idx, TYPE *result_data, + idx_t result_idx, idx_t size) { + D_ASSERT(size == 3); + + auto l_child_idx = l_idx * size; + auto r_child_idx = r_idx * size; + auto res_child_idx = result_idx * size; + + auto lx = l_data[l_child_idx + 0]; + auto ly = l_data[l_child_idx + 1]; + auto lz = l_data[l_child_idx + 2]; + + auto rx = r_data[r_child_idx + 0]; + auto ry = r_data[r_child_idx + 1]; + auto rz = r_data[r_child_idx + 2]; + + result_data[res_child_idx + 0] = ly * rz - lz * ry; + result_data[res_child_idx + 1] = lz * rx - lx * rz; + result_data[res_child_idx + 2] = lx * ry - ly * rx; + } +}; + +//------------------------------------------------------------------------------ +// Generic Execute and Bind +//------------------------------------------------------------------------------ +// This is a generic executor function for fast binary math operations on +// real-valued arrays. Array elements are assumed to be either FLOAT or DOUBLE, +// and cannot be null. (although the array itself can be null). +// In the future we could extend this further to be truly generic and handle +// other types, unary/ternary operations and/or nulls. + +template +static inline void ArrayGenericBinaryExecute(Vector &left, Vector &right, Vector &result, idx_t size, idx_t count) { + + auto &left_child = ArrayVector::GetEntry(left); + auto &right_child = ArrayVector::GetEntry(right); + + auto &left_child_validity = FlatVector::Validity(left_child); + auto &right_child_validity = FlatVector::Validity(right_child); + + UnifiedVectorFormat left_format; + UnifiedVectorFormat right_format; + + left.ToUnifiedFormat(count, left_format); + right.ToUnifiedFormat(count, right_format); + + auto left_data = FlatVector::GetData(left_child); + auto right_data = FlatVector::GetData(right_child); + auto result_data = OP::template GetResultData(result); + + for (idx_t i = 0; i < count; i++) { + auto left_idx = left_format.sel->get_index(i); + auto right_idx = right_format.sel->get_index(i); + + if (!left_format.validity.RowIsValid(left_idx) || !right_format.validity.RowIsValid(right_idx)) { + FlatVector::SetNull(result, i, true); + continue; + } + + auto left_offset = left_idx * size; + if (!left_child_validity.CheckAllValid(left_offset + size, left_offset)) { + throw InvalidInputException(StringUtil::Format("%s: left argument can not contain NULL values", OP::NAME)); + } + + auto right_offset = right_idx * size; + if (!right_child_validity.CheckAllValid(right_offset + size, right_offset)) { + throw InvalidInputException(StringUtil::Format("%s: right argument can not contain NULL values", OP::NAME)); + } + + OP::template Operation(left_data, left_idx, right_data, right_idx, result_data, i, size); + } + + if (count == 1) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +template +static void ArrayGenericBinaryFunction(DataChunk &args, ExpressionState &, Vector &result) { + auto size = ArrayType::GetSize(args.data[0].GetType()); + auto child_type = ArrayType::GetChildType(args.data[0].GetType()); + switch (child_type.id()) { + case LogicalTypeId::DOUBLE: + ArrayGenericBinaryExecute(args.data[0], args.data[1], result, size, args.size()); + break; + case LogicalTypeId::FLOAT: + ArrayGenericBinaryExecute(args.data[0], args.data[1], result, size, args.size()); + break; + default: + throw NotImplementedException(StringUtil::Format("%s: Unsupported element type", OP::NAME)); + } +} + +template +static unique_ptr ArrayGenericBinaryBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + + // construct return type + auto &left_type = arguments[0]->return_type; + auto &right_type = arguments[1]->return_type; + + auto left_size = ArrayType::GetSize(left_type); + auto right_size = ArrayType::GetSize(right_type); + if (left_size != right_size) { + throw InvalidInputException(StringUtil::Format("%s: Array arguments must be of the same size", OP::NAME)); + } + auto size = left_size; + + auto child_type = + LogicalType::MaxLogicalType(ArrayType::GetChildType(left_type), ArrayType::GetChildType(right_type)); + if (child_type != LogicalTypeId::FLOAT && child_type != LogicalTypeId::DOUBLE) { + throw InvalidInputException( + StringUtil::Format("%s: Array arguments must be of type FLOAT or DOUBLE", OP::NAME)); + } + + // the important part here is that we resolve the array size + auto array_type = LogicalType::ARRAY(child_type, size); + + bound_function.arguments[0] = array_type; + bound_function.arguments[1] = array_type; + bound_function.return_type = child_type; + + return nullptr; +} + +template +static inline void ArrayFixedBinaryFunction(DataChunk &args, ExpressionState &, Vector &result) { + ArrayGenericBinaryExecute(args.data[0], args.data[1], result, N, args.size()); +} + +//------------------------------------------------------------------------------ +// Function Registration +//------------------------------------------------------------------------------ + +// Note: In the future we could add a wrapper with a non-type template parameter to specialize for specific array sizes +// e.g. 256, 512, 1024, 2048 etc. which may allow the compiler to vectorize the loop better. Perhaps something for an +// extension. + +ScalarFunctionSet ArrayInnerProductFun::GetFunctions() { + ScalarFunctionSet set("array_inner_product"); + // Generic array inner product function + for (auto &type : LogicalType::Real()) { + set.AddFunction(ScalarFunction({LogicalType::ARRAY(type), LogicalType::ARRAY(type)}, type, + ArrayGenericBinaryFunction, + ArrayGenericBinaryBind)); + } + return set; +} + +ScalarFunctionSet ArrayDistanceFun::GetFunctions() { + ScalarFunctionSet set("array_distance"); + // Generic array distance function + for (auto &type : LogicalType::Real()) { + set.AddFunction(ScalarFunction({LogicalType::ARRAY(type), LogicalType::ARRAY(type)}, type, + ArrayGenericBinaryFunction, ArrayGenericBinaryBind)); + } + return set; +} + +ScalarFunctionSet ArrayCosineSimilarityFun::GetFunctions() { + ScalarFunctionSet set("array_cosine_similarity"); + // Generic array cosine similarity function + for (auto &type : LogicalType::Real()) { + set.AddFunction(ScalarFunction({LogicalType::ARRAY(type), LogicalType::ARRAY(type)}, type, + ArrayGenericBinaryFunction, + ArrayGenericBinaryBind)); + } + return set; +} + +ScalarFunctionSet ArrayCrossProductFun::GetFunctions() { + ScalarFunctionSet set("array_cross_product"); + + // Generic array cross product function + auto double_arr = LogicalType::ARRAY(LogicalType::DOUBLE, 3); + set.AddFunction( + ScalarFunction({double_arr, double_arr}, double_arr, ArrayFixedBinaryFunction)); + + auto float_arr = LogicalType::ARRAY(LogicalType::FLOAT, 3); + set.AddFunction( + ScalarFunction({float_arr, float_arr}, float_arr, ArrayFixedBinaryFunction)); + return set; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/array/array_value.cpp b/src/duckdb/src/core_functions/scalar/array/array_value.cpp new file mode 100644 index 000000000..d9325e745 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/array/array_value.cpp @@ -0,0 +1,87 @@ +#include "duckdb/core_functions/scalar/array_functions.hpp" +#include "duckdb/function/scalar/nested_functions.hpp" +#include "duckdb/storage/statistics/array_stats.hpp" +#include "duckdb/planner/expression/bound_function_expression.hpp" + +namespace duckdb { + +static void ArrayValueFunction(DataChunk &args, ExpressionState &state, Vector &result) { + auto array_type = result.GetType(); + + D_ASSERT(array_type.id() == LogicalTypeId::ARRAY); + D_ASSERT(args.ColumnCount() == ArrayType::GetSize(array_type)); + + auto &child_type = ArrayType::GetChildType(array_type); + + result.SetVectorType(VectorType::CONSTANT_VECTOR); + for (idx_t i = 0; i < args.ColumnCount(); i++) { + if (args.data[i].GetVectorType() != VectorType::CONSTANT_VECTOR) { + result.SetVectorType(VectorType::FLAT_VECTOR); + } + } + + auto num_rows = args.size(); + auto num_columns = args.ColumnCount(); + + auto &child = ArrayVector::GetEntry(result); + + if (num_columns > 1) { + // Ensure that the child has a validity mask of the correct size + // The SetValue call below expects the validity mask to be initialized + auto &child_validity = FlatVector::Validity(child); + child_validity.Resize(num_rows, num_rows * num_columns); + } + + for (idx_t i = 0; i < num_rows; i++) { + for (idx_t j = 0; j < num_columns; j++) { + auto val = args.GetValue(j, i).DefaultCastAs(child_type); + child.SetValue((i * num_columns) + j, val); + } + } + + result.Verify(args.size()); +} + +static unique_ptr ArrayValueBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + if (arguments.empty()) { + throw InvalidInputException("array_value requires at least one argument"); + } + + // construct return type + LogicalType child_type = arguments[0]->return_type; + for (idx_t i = 1; i < arguments.size(); i++) { + child_type = LogicalType::MaxLogicalType(child_type, arguments[i]->return_type); + } + + if (arguments.size() > ArrayType::MAX_ARRAY_SIZE) { + throw OutOfRangeException("Array size exceeds maximum allowed size"); + } + + // this is more for completeness reasons + bound_function.varargs = child_type; + bound_function.return_type = LogicalType::ARRAY(child_type, arguments.size()); + return make_uniq(bound_function.return_type); +} + +unique_ptr ArrayValueStats(ClientContext &context, FunctionStatisticsInput &input) { + auto &child_stats = input.child_stats; + auto &expr = input.expr; + auto list_stats = ArrayStats::CreateEmpty(expr.return_type); + auto &list_child_stats = ArrayStats::GetChildStats(list_stats); + for (idx_t i = 0; i < child_stats.size(); i++) { + list_child_stats.Merge(child_stats[i]); + } + return list_stats.ToUnique(); +} + +ScalarFunction ArrayValueFun::GetFunction() { + // the arguments and return types are actually set in the binder function + ScalarFunction fun("array_value", {}, LogicalTypeId::ARRAY, ArrayValueFunction, ArrayValueBind, nullptr, + ArrayValueStats); + fun.varargs = LogicalType::ANY; + fun.null_handling = FunctionNullHandling::SPECIAL_HANDLING; + return fun; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/date/to_interval.cpp b/src/duckdb/src/core_functions/scalar/date/to_interval.cpp index 4a0ee656c..5227272b4 100644 --- a/src/duckdb/src/core_functions/scalar/date/to_interval.cpp +++ b/src/duckdb/src/core_functions/scalar/date/to_interval.cpp @@ -1,7 +1,6 @@ #include "duckdb/core_functions/scalar/date_functions.hpp" #include "duckdb/common/types/interval.hpp" #include "duckdb/common/operator/multiply.hpp" -#include "duckdb/core_functions/to_interval.hpp" namespace duckdb { @@ -69,6 +68,20 @@ struct ToMinutesOperator { } }; +struct ToSecondsOperator { + template + static inline TR Operation(TA input) { + interval_t result; + result.months = 0; + result.days = 0; + if (!TryMultiplyOperator::Operation(input, Interval::MICROS_PER_SEC, + result.micros)) { + throw OutOfRangeException("Interval value %d seconds out of range", input); + } + return result; + } +}; + struct ToMilliSecondsOperator { template static inline TR Operation(TA input) { diff --git a/src/duckdb/src/core_functions/scalar/list/array_slice.cpp b/src/duckdb/src/core_functions/scalar/list/array_slice.cpp index a9b2aeecb..725846d17 100644 --- a/src/duckdb/src/core_functions/scalar/list/array_slice.cpp +++ b/src/duckdb/src/core_functions/scalar/list/array_slice.cpp @@ -6,6 +6,7 @@ #include "duckdb/function/scalar/string_functions.hpp" #include "duckdb/planner/expression/bound_function_expression.hpp" #include "duckdb/planner/expression/bound_constant_expression.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" namespace duckdb { @@ -379,6 +380,13 @@ static unique_ptr ArraySliceBind(ClientContext &context, ScalarFun D_ASSERT(bound_function.arguments.size() == 3 || bound_function.arguments.size() == 4); switch (arguments[0]->return_type.id()) { + case LogicalTypeId::ARRAY: { + // Cast to list + auto child_type = ArrayType::GetChildType(arguments[0]->return_type); + auto target_type = LogicalType::LIST(child_type); + arguments[0] = BoundCastExpression::AddCastToType(context, std::move(arguments[0]), target_type); + bound_function.return_type = arguments[0]->return_type; + } break; case LogicalTypeId::LIST: // The result is the same type bound_function.return_type = arguments[0]->return_type; diff --git a/src/duckdb/src/core_functions/scalar/list/list_aggregates.cpp b/src/duckdb/src/core_functions/scalar/list/list_aggregates.cpp index 0f561b1ff..4539b4e1f 100644 --- a/src/duckdb/src/core_functions/scalar/list/list_aggregates.cpp +++ b/src/duckdb/src/core_functions/scalar/list/list_aggregates.cpp @@ -7,6 +7,7 @@ #include "duckdb/planner/expression/bound_aggregate_expression.hpp" #include "duckdb/planner/expression/bound_constant_expression.hpp" #include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" #include "duckdb/planner/expression_binder.hpp" #include "duckdb/function/function_binder.hpp" @@ -47,13 +48,13 @@ struct ListAggregatesBindData : public FunctionData { return result; } - static void Serialize(Serializer &serializer, const optional_ptr bind_data_p, - const ScalarFunction &function) { + static void SerializeFunction(Serializer &serializer, const optional_ptr bind_data_p, + const ScalarFunction &function) { auto bind_data = dynamic_cast(bind_data_p.get()); serializer.WritePropertyWithDefault(100, "bind_data", bind_data, (const ListAggregatesBindData *)nullptr); } - static unique_ptr Deserialize(Deserializer &deserializer, ScalarFunction &bound_function) { + static unique_ptr DeserializeFunction(Deserializer &deserializer, ScalarFunction &bound_function) { auto result = deserializer.ReadPropertyWithDefault>( 100, "bind_data", unique_ptr(nullptr)); if (!result) { @@ -421,12 +422,24 @@ ListAggregatesBindFunction(ClientContext &context, ScalarFunction &bound_functio template static unique_ptr ListAggregatesBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { + + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + if (arguments[0]->return_type.id() == LogicalTypeId::SQLNULL) { return ListAggregatesBindFailure(bound_function); } bool is_parameter = arguments[0]->return_type.id() == LogicalTypeId::UNKNOWN; - auto list_child_type = is_parameter ? LogicalTypeId::UNKNOWN : ListType::GetChildType(arguments[0]->return_type); + LogicalType child_type; + if (is_parameter) { + child_type = LogicalType::ANY; + } else if (arguments[0]->return_type.id() == LogicalTypeId::LIST || + arguments[0]->return_type.id() == LogicalTypeId::MAP) { + child_type = ListType::GetChildType(arguments[0]->return_type); + } else { + // Unreachable + throw InvalidInputException("First argument of list aggregate must be a list, map or array"); + } string function_name = "histogram"; if (IS_AGGR) { // get the name of the aggregate function @@ -453,7 +466,7 @@ static unique_ptr ListAggregatesBind(ClientContext &context, Scala // find a matching aggregate function string error; vector types; - types.push_back(list_child_type); + types.push_back(child_type); // push any extra arguments into the type list for (idx_t i = 2; i < arguments.size(); i++) { types.push_back(arguments[i]->return_type); @@ -468,14 +481,14 @@ static unique_ptr ListAggregatesBind(ClientContext &context, Scala // found a matching function, bind it as an aggregate auto best_function = func.functions.GetFunctionByOffset(best_function_idx); if (IS_AGGR) { - return ListAggregatesBindFunction(context, bound_function, list_child_type, best_function, arguments); + return ListAggregatesBindFunction(context, bound_function, child_type, best_function, arguments); } // create the unordered map histogram function D_ASSERT(best_function.arguments.size() == 1); auto key_type = best_function.arguments[0]; auto aggr_function = HistogramFun::GetHistogramUnorderedMap(key_type); - return ListAggregatesBindFunction(context, bound_function, list_child_type, aggr_function, arguments); + return ListAggregatesBindFunction(context, bound_function, child_type, aggr_function, arguments); } static unique_ptr ListAggregateBind(ClientContext &context, ScalarFunction &bound_function, @@ -493,6 +506,8 @@ static unique_ptr ListDistinctBind(ClientContext &context, ScalarF D_ASSERT(bound_function.arguments.size() == 1); D_ASSERT(arguments.size() == 1); + + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); bound_function.return_type = arguments[0]->return_type; return ListAggregatesBind<>(context, bound_function, arguments); @@ -513,8 +528,8 @@ ScalarFunction ListAggregateFun::GetFunction() { ListAggregateFunction, ListAggregateBind); result.null_handling = FunctionNullHandling::SPECIAL_HANDLING; result.varargs = LogicalType::ANY; - result.serialize = ListAggregatesBindData::Serialize; - result.deserialize = ListAggregatesBindData::Deserialize; + result.serialize = ListAggregatesBindData::SerializeFunction; + result.deserialize = ListAggregatesBindData::DeserializeFunction; return result; } diff --git a/src/duckdb/src/core_functions/scalar/list/list_filter.cpp b/src/duckdb/src/core_functions/scalar/list/list_filter.cpp new file mode 100644 index 000000000..9dbab0981 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/list/list_filter.cpp @@ -0,0 +1,49 @@ +#include "duckdb/core_functions/scalar/list_functions.hpp" + +#include "duckdb/core_functions/lambda_functions.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" + +namespace duckdb { + +static unique_ptr ListFilterBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + + // the list column and the bound lambda expression + D_ASSERT(arguments.size() == 2); + if (arguments[1]->expression_class != ExpressionClass::BOUND_LAMBDA) { + throw BinderException("Invalid lambda expression!"); + } + + auto &bound_lambda_expr = arguments[1]->Cast(); + + // try to cast to boolean, if the return type of the lambda filter expression is not already boolean + if (bound_lambda_expr.lambda_expr->return_type != LogicalType::BOOLEAN) { + auto cast_lambda_expr = + BoundCastExpression::AddCastToType(context, std::move(bound_lambda_expr.lambda_expr), LogicalType::BOOLEAN); + bound_lambda_expr.lambda_expr = std::move(cast_lambda_expr); + } + + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + + bound_function.return_type = arguments[0]->return_type; + auto has_index = bound_lambda_expr.parameter_count == 2; + return LambdaFunctions::ListLambdaBind(context, bound_function, arguments, has_index); +} + +static LogicalType ListFilterBindLambda(const idx_t parameter_idx, const LogicalType &list_child_type) { + return LambdaFunctions::BindBinaryLambda(parameter_idx, list_child_type); +} + +ScalarFunction ListFilterFun::GetFunction() { + ScalarFunction fun({LogicalType::LIST(LogicalType::ANY), LogicalType::LAMBDA}, LogicalType::LIST(LogicalType::ANY), + LambdaFunctions::ListFilterFunction, ListFilterBind, nullptr, nullptr); + + fun.null_handling = FunctionNullHandling::SPECIAL_HANDLING; + fun.serialize = ListLambdaBindData::Serialize; + fun.deserialize = ListLambdaBindData::Deserialize; + fun.bind_lambda = ListFilterBindLambda; + + return fun; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/list/list_lambdas.cpp b/src/duckdb/src/core_functions/scalar/list/list_lambdas.cpp deleted file mode 100644 index a9a8feba3..000000000 --- a/src/duckdb/src/core_functions/scalar/list/list_lambdas.cpp +++ /dev/null @@ -1,412 +0,0 @@ -#include "duckdb/core_functions/scalar/list_functions.hpp" -#include "duckdb/planner/expression/bound_function_expression.hpp" -#include "duckdb/function/scalar/nested_functions.hpp" -#include "duckdb/planner/expression_iterator.hpp" -#include "duckdb/planner/expression/bound_reference_expression.hpp" -#include "duckdb/execution/expression_executor.hpp" -#include "duckdb/planner/expression/bound_constant_expression.hpp" -#include "duckdb/planner/expression/bound_lambda_expression.hpp" -#include "duckdb/planner/expression/bound_cast_expression.hpp" -#include "duckdb/function/cast/cast_function_set.hpp" -#include "duckdb/common/serializer/serializer.hpp" -#include "duckdb/common/serializer/deserializer.hpp" - -namespace duckdb { - -struct ListLambdaBindData : public FunctionData { - ListLambdaBindData(const LogicalType &stype_p, unique_ptr lambda_expr); - ~ListLambdaBindData() override; - - LogicalType stype; - unique_ptr lambda_expr; - -public: - bool Equals(const FunctionData &other_p) const override; - unique_ptr Copy() const override; - - static void Serialize(Serializer &serializer, const optional_ptr bind_data_p, - const ScalarFunction &function) { - // auto &bind_data = bind_data_p->Cast(); - // serializer.WriteProperty(100, "stype", bind_data.stype); - // serializer.WritePropertyWithDefault(101, "lambda_expr", bind_data.lambda_expr, - // unique_ptr()); - throw NotImplementedException("FIXME: list lambda serialize"); - } - - static unique_ptr Deserialize(Deserializer &deserializer, ScalarFunction &function) { - // auto stype = deserializer.ReadProperty(100, "stype"); - // auto lambda_expr = - // deserializer.ReadPropertyWithDefault>(101, "lambda_expr", - // unique_ptr()); return make_uniq(stype, std::move(lambda_expr)); - throw NotImplementedException("FIXME: list lambda deserialize"); - } -}; - -ListLambdaBindData::ListLambdaBindData(const LogicalType &stype_p, unique_ptr lambda_expr_p) - : stype(stype_p), lambda_expr(std::move(lambda_expr_p)) { -} - -unique_ptr ListLambdaBindData::Copy() const { - return make_uniq(stype, lambda_expr ? lambda_expr->Copy() : nullptr); -} - -bool ListLambdaBindData::Equals(const FunctionData &other_p) const { - auto &other = other_p.Cast(); - return Expression::Equals(lambda_expr, other.lambda_expr) && stype == other.stype; -} - -ListLambdaBindData::~ListLambdaBindData() { -} - -static void AppendTransformedToResult(Vector &lambda_vector, idx_t &elem_cnt, Vector &result) { - - // append the lambda_vector to the result list - UnifiedVectorFormat lambda_child_data; - lambda_vector.ToUnifiedFormat(elem_cnt, lambda_child_data); - ListVector::Append(result, lambda_vector, *lambda_child_data.sel, elem_cnt, 0); -} - -static void AppendFilteredToResult(Vector &lambda_vector, list_entry_t *result_entries, idx_t &elem_cnt, Vector &result, - idx_t &curr_list_len, idx_t &curr_list_offset, idx_t &appended_lists_cnt, - vector &lists_len, idx_t &curr_original_list_len, DataChunk &input_chunk) { - - idx_t true_count = 0; - SelectionVector true_sel(elem_cnt); - UnifiedVectorFormat lambda_data; - lambda_vector.ToUnifiedFormat(elem_cnt, lambda_data); - - auto lambda_values = UnifiedVectorFormat::GetData(lambda_data); - auto &lambda_validity = lambda_data.validity; - - // compute the new lengths and offsets, and create a selection vector - for (idx_t i = 0; i < elem_cnt; i++) { - auto entry = lambda_data.sel->get_index(i); - - while (appended_lists_cnt < lists_len.size() && lists_len[appended_lists_cnt] == 0) { - result_entries[appended_lists_cnt].offset = curr_list_offset; - result_entries[appended_lists_cnt].length = 0; - appended_lists_cnt++; - } - - // found a true value - if (lambda_validity.RowIsValid(entry) && lambda_values[entry]) { - true_sel.set_index(true_count++, i); - curr_list_len++; - } - - curr_original_list_len++; - - if (lists_len[appended_lists_cnt] == curr_original_list_len) { - result_entries[appended_lists_cnt].offset = curr_list_offset; - result_entries[appended_lists_cnt].length = curr_list_len; - curr_list_offset += curr_list_len; - appended_lists_cnt++; - curr_list_len = 0; - curr_original_list_len = 0; - } - } - - while (appended_lists_cnt < lists_len.size() && lists_len[appended_lists_cnt] == 0) { - result_entries[appended_lists_cnt].offset = curr_list_offset; - result_entries[appended_lists_cnt].length = 0; - appended_lists_cnt++; - } - - // slice to get the new lists and append them to the result - Vector new_lists(input_chunk.data[0], true_sel, true_count); - new_lists.Flatten(true_count); - UnifiedVectorFormat new_lists_child_data; - new_lists.ToUnifiedFormat(true_count, new_lists_child_data); - ListVector::Append(result, new_lists, *new_lists_child_data.sel, true_count, 0); -} - -static void ExecuteExpression(vector &types, vector &result_types, idx_t &elem_cnt, - SelectionVector &sel, vector &sel_vectors, DataChunk &input_chunk, - DataChunk &lambda_chunk, Vector &child_vector, DataChunk &args, - ExpressionExecutor &expr_executor) { - - input_chunk.SetCardinality(elem_cnt); - lambda_chunk.SetCardinality(elem_cnt); - - // set the list child vector - Vector slice(child_vector, sel, elem_cnt); - Vector second_slice(child_vector, sel, elem_cnt); - slice.Flatten(elem_cnt); - second_slice.Flatten(elem_cnt); - - input_chunk.data[0].Reference(slice); - input_chunk.data[1].Reference(second_slice); - - // set the other vectors - vector slices; - for (idx_t col_idx = 0; col_idx < args.ColumnCount() - 1; col_idx++) { - slices.emplace_back(args.data[col_idx + 1], sel_vectors[col_idx], elem_cnt); - slices[col_idx].Flatten(elem_cnt); - input_chunk.data[col_idx + 2].Reference(slices[col_idx]); - } - - // execute the lambda expression - expr_executor.Execute(input_chunk, lambda_chunk); -} - -template -static void ListLambdaFunction(DataChunk &args, ExpressionState &state, Vector &result) { - - // always at least the list argument - D_ASSERT(args.ColumnCount() >= 1); - - auto count = args.size(); - Vector &lists = args.data[0]; - - result.SetVectorType(VectorType::FLAT_VECTOR); - auto result_entries = FlatVector::GetData(result); - auto &result_validity = FlatVector::Validity(result); - - if (lists.GetType().id() == LogicalTypeId::SQLNULL) { - result_validity.SetInvalid(0); - return; - } - - // e.g. window functions in sub queries return dictionary vectors, which segfault on expression execution - // if not flattened first - for (idx_t i = 1; i < args.ColumnCount(); i++) { - if (args.data[i].GetVectorType() != VectorType::FLAT_VECTOR && - args.data[i].GetVectorType() != VectorType::CONSTANT_VECTOR) { - args.data[i].Flatten(count); - } - } - - // get the lists data - UnifiedVectorFormat lists_data; - lists.ToUnifiedFormat(count, lists_data); - auto list_entries = UnifiedVectorFormat::GetData(lists_data); - - // get the lambda expression - auto &func_expr = state.expr.Cast(); - auto &info = func_expr.bind_info->Cast(); - auto &lambda_expr = info.lambda_expr; - - // get the child vector and child data - auto lists_size = ListVector::GetListSize(lists); - auto &child_vector = ListVector::GetEntry(lists); - child_vector.Flatten(lists_size); - UnifiedVectorFormat child_data; - child_vector.ToUnifiedFormat(lists_size, child_data); - - // to slice the child vector - SelectionVector sel(STANDARD_VECTOR_SIZE); - - // this vector never contains more than one element - vector result_types; - result_types.push_back(lambda_expr->return_type); - - // non-lambda parameter columns - vector columns; - vector indexes; - vector sel_vectors; - - vector types; - types.push_back(child_vector.GetType()); - types.push_back(child_vector.GetType()); - - // skip the list column - for (idx_t i = 1; i < args.ColumnCount(); i++) { - columns.emplace_back(); - args.data[i].ToUnifiedFormat(count, columns[i - 1]); - indexes.push_back(0); - sel_vectors.emplace_back(STANDARD_VECTOR_SIZE); - types.push_back(args.data[i].GetType()); - } - - // get the expression executor - ExpressionExecutor expr_executor(state.GetContext(), *lambda_expr); - - // these are only for the list_filter - vector lists_len; - idx_t curr_list_len = 0; - idx_t curr_list_offset = 0; - idx_t appended_lists_cnt = 0; - idx_t curr_original_list_len = 0; - - if (!IS_TRANSFORM) { - lists_len.reserve(count); - } - - DataChunk input_chunk; - DataChunk lambda_chunk; - input_chunk.InitializeEmpty(types); - lambda_chunk.Initialize(Allocator::DefaultAllocator(), result_types); - - // loop over the child entries and create chunks to be executed by the expression executor - idx_t elem_cnt = 0; - idx_t offset = 0; - for (idx_t row_idx = 0; row_idx < count; row_idx++) { - - auto lists_index = lists_data.sel->get_index(row_idx); - const auto &list_entry = list_entries[lists_index]; - - // set the result to NULL for this row - if (!lists_data.validity.RowIsValid(lists_index)) { - result_validity.SetInvalid(row_idx); - if (!IS_TRANSFORM) { - lists_len.push_back(0); - } - continue; - } - - // set the length and offset of the resulting lists of list_transform - if (IS_TRANSFORM) { - result_entries[row_idx].offset = offset; - result_entries[row_idx].length = list_entry.length; - offset += list_entry.length; - } else { - lists_len.push_back(list_entry.length); - } - - // empty list, nothing to execute - if (list_entry.length == 0) { - continue; - } - - // get the data indexes - for (idx_t col_idx = 0; col_idx < args.ColumnCount() - 1; col_idx++) { - indexes[col_idx] = columns[col_idx].sel->get_index(row_idx); - } - - // iterate list elements and create transformed expression columns - for (idx_t child_idx = 0; child_idx < list_entry.length; child_idx++) { - // reached STANDARD_VECTOR_SIZE elements - if (elem_cnt == STANDARD_VECTOR_SIZE) { - lambda_chunk.Reset(); - ExecuteExpression(types, result_types, elem_cnt, sel, sel_vectors, input_chunk, lambda_chunk, - child_vector, args, expr_executor); - - auto &lambda_vector = lambda_chunk.data[0]; - - if (IS_TRANSFORM) { - AppendTransformedToResult(lambda_vector, elem_cnt, result); - } else { - AppendFilteredToResult(lambda_vector, result_entries, elem_cnt, result, curr_list_len, - curr_list_offset, appended_lists_cnt, lists_len, curr_original_list_len, - input_chunk); - } - elem_cnt = 0; - } - - // to slice the child vector - auto source_idx = child_data.sel->get_index(list_entry.offset + child_idx); - sel.set_index(elem_cnt, source_idx); - - // for each column, set the index of the selection vector to slice properly - for (idx_t col_idx = 0; col_idx < args.ColumnCount() - 1; col_idx++) { - sel_vectors[col_idx].set_index(elem_cnt, indexes[col_idx]); - } - elem_cnt++; - } - } - - lambda_chunk.Reset(); - ExecuteExpression(types, result_types, elem_cnt, sel, sel_vectors, input_chunk, lambda_chunk, child_vector, args, - expr_executor); - auto &lambda_vector = lambda_chunk.data[0]; - - if (IS_TRANSFORM) { - AppendTransformedToResult(lambda_vector, elem_cnt, result); - } else { - AppendFilteredToResult(lambda_vector, result_entries, elem_cnt, result, curr_list_len, curr_list_offset, - appended_lists_cnt, lists_len, curr_original_list_len, input_chunk); - } - - if (args.AllConstant()) { - result.SetVectorType(VectorType::CONSTANT_VECTOR); - } -} - -static void ListTransformFunction(DataChunk &args, ExpressionState &state, Vector &result) { - ListLambdaFunction<>(args, state, result); -} - -static void ListFilterFunction(DataChunk &args, ExpressionState &state, Vector &result) { - ListLambdaFunction(args, state, result); -} - -template -static unique_ptr ListLambdaBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments) { - auto &bound_lambda_expr = arguments[1]->Cast(); - if (bound_lambda_expr.parameter_count != LAMBDA_PARAM_CNT) { - throw BinderException("Incorrect number of parameters in lambda function! " + bound_function.name + - " expects " + to_string(LAMBDA_PARAM_CNT) + " parameter(s)."); - } - - if (arguments[0]->return_type.id() == LogicalTypeId::SQLNULL) { - bound_function.arguments[0] = LogicalType::SQLNULL; - bound_function.return_type = LogicalType::SQLNULL; - return make_uniq(bound_function.return_type, nullptr); - } - - if (arguments[0]->return_type.id() == LogicalTypeId::UNKNOWN) { - throw ParameterNotResolvedException(); - } - - D_ASSERT(arguments[0]->return_type.id() == LogicalTypeId::LIST); - - // get the lambda expression and put it in the bind info - auto lambda_expr = std::move(bound_lambda_expr.lambda_expr); - return make_uniq(bound_function.return_type, std::move(lambda_expr)); -} - -static unique_ptr ListTransformBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments) { - - // at least the list column and the lambda function - D_ASSERT(arguments.size() == 2); - if (arguments[1]->expression_class != ExpressionClass::BOUND_LAMBDA) { - throw BinderException("Invalid lambda expression!"); - } - - auto &bound_lambda_expr = arguments[1]->Cast(); - bound_function.return_type = LogicalType::LIST(bound_lambda_expr.lambda_expr->return_type); - return ListLambdaBind<1>(context, bound_function, arguments); -} - -static unique_ptr ListFilterBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments) { - - // at least the list column and the lambda function - D_ASSERT(arguments.size() == 2); - if (arguments[1]->expression_class != ExpressionClass::BOUND_LAMBDA) { - throw BinderException("Invalid lambda expression!"); - } - - // try to cast to boolean, if the return type of the lambda filter expression is not already boolean - auto &bound_lambda_expr = arguments[1]->Cast(); - if (bound_lambda_expr.lambda_expr->return_type != LogicalType::BOOLEAN) { - auto cast_lambda_expr = - BoundCastExpression::AddCastToType(context, std::move(bound_lambda_expr.lambda_expr), LogicalType::BOOLEAN); - bound_lambda_expr.lambda_expr = std::move(cast_lambda_expr); - } - - bound_function.return_type = arguments[0]->return_type; - return ListLambdaBind<1>(context, bound_function, arguments); -} - -ScalarFunction ListTransformFun::GetFunction() { - ScalarFunction fun({LogicalType::LIST(LogicalType::ANY), LogicalType::LAMBDA}, LogicalType::LIST(LogicalType::ANY), - ListTransformFunction, ListTransformBind, nullptr, nullptr); - fun.null_handling = FunctionNullHandling::SPECIAL_HANDLING; - fun.serialize = ListLambdaBindData::Serialize; - fun.deserialize = ListLambdaBindData::Deserialize; - return fun; -} - -ScalarFunction ListFilterFun::GetFunction() { - ScalarFunction fun({LogicalType::LIST(LogicalType::ANY), LogicalType::LAMBDA}, LogicalType::LIST(LogicalType::ANY), - ListFilterFunction, ListFilterBind, nullptr, nullptr); - fun.null_handling = FunctionNullHandling::SPECIAL_HANDLING; - fun.serialize = ListLambdaBindData::Serialize; - fun.deserialize = ListLambdaBindData::Deserialize; - return fun; -} - -} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/list/list_sort.cpp b/src/duckdb/src/core_functions/scalar/list/list_sort.cpp index 02dc205ea..340ffc5d8 100644 --- a/src/duckdb/src/core_functions/scalar/list/list_sort.cpp +++ b/src/duckdb/src/core_functions/scalar/list/list_sort.cpp @@ -2,6 +2,7 @@ #include "duckdb/common/enum_util.hpp" #include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/planner/expression/bound_reference_expression.hpp" #include "duckdb/main/config.hpp" @@ -10,14 +11,15 @@ namespace duckdb { struct ListSortBindData : public FunctionData { - ListSortBindData(OrderType order_type_p, OrderByNullType null_order_p, const LogicalType &return_type_p, - const LogicalType &child_type_p, ClientContext &context_p); + ListSortBindData(OrderType order_type_p, OrderByNullType null_order_p, bool is_grade_up, + const LogicalType &return_type_p, const LogicalType &child_type_p, ClientContext &context_p); ~ListSortBindData() override; OrderType order_type; OrderByNullType null_order; LogicalType return_type; LogicalType child_type; + bool is_grade_up; vector types; vector payload_types; @@ -31,11 +33,11 @@ struct ListSortBindData : public FunctionData { unique_ptr Copy() const override; }; -ListSortBindData::ListSortBindData(OrderType order_type_p, OrderByNullType null_order_p, +ListSortBindData::ListSortBindData(OrderType order_type_p, OrderByNullType null_order_p, bool is_grade_up_p, const LogicalType &return_type_p, const LogicalType &child_type_p, ClientContext &context_p) : order_type(order_type_p), null_order(null_order_p), return_type(return_type_p), child_type(child_type_p), - context(context_p) { + is_grade_up(is_grade_up_p), context(context_p) { // get the vector types types.emplace_back(LogicalType::USMALLINT); @@ -57,12 +59,12 @@ ListSortBindData::ListSortBindData(OrderType order_type_p, OrderByNullType null_ } unique_ptr ListSortBindData::Copy() const { - return make_uniq(order_type, null_order, return_type, child_type, context); + return make_uniq(order_type, null_order, is_grade_up, return_type, child_type, context); } bool ListSortBindData::Equals(const FunctionData &other_p) const { auto &other = other_p.Cast(); - return order_type == other.order_type && null_order == other.null_order; + return order_type == other.order_type && null_order == other.null_order && is_grade_up == other.is_grade_up; } ListSortBindData::~ListSortBindData() { @@ -120,18 +122,20 @@ static void ListSortFunction(DataChunk &args, ExpressionState &state, Vector &re LocalSortState local_sort_state; local_sort_state.Initialize(global_sort_state, buffer_manager); + Vector sort_result_vec = info.is_grade_up ? Vector(input_lists.GetType()) : result; + // this ensures that we do not change the order of the entries in the input chunk - VectorOperations::Copy(input_lists, result, count, 0, 0); + VectorOperations::Copy(input_lists, sort_result_vec, count, 0, 0); // get the child vector - auto lists_size = ListVector::GetListSize(result); - auto &child_vector = ListVector::GetEntry(result); + auto lists_size = ListVector::GetListSize(sort_result_vec); + auto &child_vector = ListVector::GetEntry(sort_result_vec); UnifiedVectorFormat child_data; child_vector.ToUnifiedFormat(lists_size, child_data); // get the lists data UnifiedVectorFormat lists_data; - result.ToUnifiedFormat(count, lists_data); + sort_result_vec.ToUnifiedFormat(count, lists_data); auto list_entries = UnifiedVectorFormat::GetData(lists_data); // create the lists_indices vector, this contains an element for each list's entry, @@ -191,6 +195,13 @@ static void ListSortFunction(DataChunk &args, ExpressionState &state, Vector &re local_sort_state, data_to_sort, lists_indices); } + if (info.is_grade_up) { + ListVector::Reserve(result, lists_size); + ListVector::SetListSize(result, lists_size); + auto result_data = ListVector::GetData(result); + memcpy(result_data, list_entries, count * sizeof(list_entry_t)); + } + if (data_to_sort) { // add local state to global state, which sorts the data global_sort_state.AddLocalState(local_sort_state); @@ -224,8 +235,19 @@ static void ListSortFunction(DataChunk &args, ExpressionState &state, Vector &re } D_ASSERT(sel_sorted_idx == incr_payload_count); - child_vector.Slice(sel_sorted, sel_sorted_idx); - child_vector.Flatten(sel_sorted_idx); + if (info.is_grade_up) { + auto &result_entry = ListVector::GetEntry(result); + auto result_data = ListVector::GetData(result); + for (idx_t i = 0; i < count; i++) { + for (idx_t j = result_data[i].offset; j < result_data[i].offset + result_data[i].length; j++) { + auto b = sel_sorted.get_index(j) - result_data[i].offset; + result_entry.SetValue(j, Value::BIGINT(b + 1)); + } + } + } else { + child_vector.Slice(sel_sorted, sel_sorted_idx); + child_vector.Flatten(sel_sorted_idx); + } } if (args.AllConstant()) { @@ -242,14 +264,16 @@ static unique_ptr ListSortBind(ClientContext &context, ScalarFunct bound_function.arguments[0] = LogicalTypeId::UNKNOWN; bound_function.return_type = LogicalType::SQLNULL; child_type = bound_function.return_type; - return make_uniq(order, null_order, bound_function.return_type, child_type, context); + return make_uniq(order, null_order, false, bound_function.return_type, child_type, context); } + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + child_type = ListType::GetChildType(arguments[0]->return_type); + bound_function.arguments[0] = arguments[0]->return_type; bound_function.return_type = arguments[0]->return_type; - child_type = ListType::GetChildType(arguments[0]->return_type); - return make_uniq(order, null_order, bound_function.return_type, child_type, context); + return make_uniq(order, null_order, false, bound_function.return_type, child_type, context); } template @@ -262,6 +286,31 @@ static T GetOrder(ClientContext &context, Expression &expr) { return EnumUtil::FromString(order_name.c_str()); } +static unique_ptr ListGradeUpBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + + D_ASSERT(!arguments.empty() && arguments.size() <= 3); + auto order = OrderType::ORDER_DEFAULT; + auto null_order = OrderByNullType::ORDER_DEFAULT; + + // get the sorting order + if (arguments.size() >= 2) { + order = GetOrder(context, *arguments[1]); + } + // get the null sorting order + if (arguments.size() == 3) { + null_order = GetOrder(context, *arguments[2]); + } + auto &config = DBConfig::GetConfig(context); + order = config.ResolveOrder(order); + null_order = config.ResolveNullOrder(order, null_order); + + bound_function.arguments[0] = arguments[0]->return_type; + bound_function.return_type = LogicalType::LIST(LogicalTypeId::BIGINT); + auto child_type = ListType::GetChildType(arguments[0]->return_type); + return make_uniq(order, null_order, true, bound_function.return_type, child_type, context); +} + static unique_ptr ListNormalSortBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { D_ASSERT(!arguments.empty() && arguments.size() <= 3); @@ -326,6 +375,26 @@ ScalarFunctionSet ListSortFun::GetFunctions() { return list_sort; } +ScalarFunctionSet ListGradeUpFun::GetFunctions() { + // one parameter: list + ScalarFunction sort({LogicalType::LIST(LogicalType::ANY)}, LogicalType::LIST(LogicalType::ANY), ListSortFunction, + ListGradeUpBind); + + // two parameters: list, order + ScalarFunction sort_order({LogicalType::LIST(LogicalType::ANY), LogicalType::VARCHAR}, + LogicalType::LIST(LogicalType::ANY), ListSortFunction, ListGradeUpBind); + + // three parameters: list, order, null order + ScalarFunction sort_orders({LogicalType::LIST(LogicalType::ANY), LogicalType::VARCHAR, LogicalType::VARCHAR}, + LogicalType::LIST(LogicalType::ANY), ListSortFunction, ListGradeUpBind); + + ScalarFunctionSet list_grade_up; + list_grade_up.AddFunction(sort); + list_grade_up.AddFunction(sort_order); + list_grade_up.AddFunction(sort_orders); + return list_grade_up; +} + ScalarFunctionSet ListReverseSortFun::GetFunctions() { // one parameter: list ScalarFunction sort_reverse({LogicalType::LIST(LogicalType::ANY)}, LogicalType::LIST(LogicalType::ANY), diff --git a/src/duckdb/src/core_functions/scalar/list/list_transform.cpp b/src/duckdb/src/core_functions/scalar/list/list_transform.cpp new file mode 100644 index 000000000..01f10641e --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/list/list_transform.cpp @@ -0,0 +1,41 @@ +#include "duckdb/core_functions/scalar/list_functions.hpp" + +#include "duckdb/core_functions/lambda_functions.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" + +namespace duckdb { + +static unique_ptr ListTransformBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + + // the list column and the bound lambda expression + D_ASSERT(arguments.size() == 2); + if (arguments[1]->expression_class != ExpressionClass::BOUND_LAMBDA) { + throw BinderException("Invalid lambda expression!"); + } + + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + + auto &bound_lambda_expr = arguments[1]->Cast(); + bound_function.return_type = LogicalType::LIST(bound_lambda_expr.lambda_expr->return_type); + auto has_index = bound_lambda_expr.parameter_count == 2; + return LambdaFunctions::ListLambdaBind(context, bound_function, arguments, has_index); +} + +static LogicalType ListTransformBindLambda(const idx_t parameter_idx, const LogicalType &list_child_type) { + return LambdaFunctions::BindBinaryLambda(parameter_idx, list_child_type); +} + +ScalarFunction ListTransformFun::GetFunction() { + ScalarFunction fun({LogicalType::LIST(LogicalType::ANY), LogicalType::LAMBDA}, LogicalType::LIST(LogicalType::ANY), + LambdaFunctions::ListTransformFunction, ListTransformBind, nullptr, nullptr); + + fun.null_handling = FunctionNullHandling::SPECIAL_HANDLING; + fun.serialize = ListLambdaBindData::Serialize; + fun.deserialize = ListLambdaBindData::Deserialize; + fun.bind_lambda = ListTransformBindLambda; + + return fun; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/math/numeric.cpp b/src/duckdb/src/core_functions/scalar/math/numeric.cpp index 740b9ed4a..19c841b2e 100644 --- a/src/duckdb/src/core_functions/scalar/math/numeric.cpp +++ b/src/duckdb/src/core_functions/scalar/math/numeric.cpp @@ -816,29 +816,6 @@ ScalarFunction Log10Fun::GetFunction() { ScalarFunction::UnaryFunction); } -//===--------------------------------------------------------------------===// -// log with base -//===--------------------------------------------------------------------===// -struct LogBaseOperator { - template - static inline TR Operation(TA b, TB x) { - auto divisor = Log10Operator::Operation(b); - if (divisor == 0) { - throw OutOfRangeException("divison by zero in based logarithm"); - } - return Log10Operator::Operation(x) / divisor; - } -}; - -ScalarFunctionSet LogFun::GetFunctions() { - ScalarFunctionSet funcs; - funcs.AddFunction(ScalarFunction({LogicalType::DOUBLE}, LogicalType::DOUBLE, - ScalarFunction::UnaryFunction)); - funcs.AddFunction(ScalarFunction({LogicalType::DOUBLE, LogicalType::DOUBLE}, LogicalType::DOUBLE, - ScalarFunction::BinaryFunction)); - return funcs; -} - //===--------------------------------------------------------------------===// // log2 //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/core_functions/scalar/string/jaccard.cpp b/src/duckdb/src/core_functions/scalar/string/jaccard.cpp index e3f081b6a..690244427 100644 --- a/src/duckdb/src/core_functions/scalar/string/jaccard.cpp +++ b/src/duckdb/src/core_functions/scalar/string/jaccard.cpp @@ -1,41 +1,48 @@ -#include "duckdb/common/map.hpp" -#include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/core_functions/scalar/string_functions.hpp" +#include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/common/map.hpp" -#include #include namespace duckdb { -namespace { -constexpr size_t MAX_SIZE = std::numeric_limits::max() + 1; -} - -static inline std::bitset GetSet(const string_t &str) { - std::bitset array_set; - +static inline map GetSet(const string_t &str) { + auto map_of_chars = map {}; idx_t str_len = str.GetSize(); auto s = str.GetData(); for (idx_t pos = 0; pos < str_len; pos++) { - array_set.set(static_cast(s[pos])); + map_of_chars.insert(std::make_pair(s[pos], 1)); } - return array_set; + return map_of_chars; } static double JaccardSimilarity(const string_t &str, const string_t &txt) { if (str.GetSize() < 1 || txt.GetSize() < 1) { throw InvalidInputException("Jaccard Function: An argument too short!"); } - std::bitset m_str, m_txt; + map m_str, m_txt; m_str = GetSet(str); m_txt = GetSet(txt); - idx_t size_intersect = (m_str & m_txt).count(); - idx_t size_union = (m_str | m_txt).count(); + if (m_str.size() > m_txt.size()) { + m_str.swap(m_txt); + } + + for (auto const &achar : m_str) { + ++m_txt[achar.first]; + } + // m_txt.size is now size of union. + + idx_t size_intersect = 0; + for (const auto &apair : m_txt) { + if (apair.second > 1) { + size_intersect++; + } + } - return static_cast(size_intersect) / static_cast(size_union); + return (double)size_intersect / (double)m_txt.size(); } static double JaccardScalarFunction(Vector &result, const string_t str, string_t tgt) { diff --git a/src/duckdb/src/execution/aggregate_hashtable.cpp b/src/duckdb/src/execution/aggregate_hashtable.cpp index d15583c2e..be5aa9681 100644 --- a/src/duckdb/src/execution/aggregate_hashtable.cpp +++ b/src/duckdb/src/execution/aggregate_hashtable.cpp @@ -328,7 +328,7 @@ idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(DataChunk &groups, V // Compute the entry in the table based on the hash using a modulo, // and precompute the hash salts for faster comparison below auto ht_offsets = FlatVector::GetData(state.ht_offsets); - const auto hash_salts = FlatVector::GetData(state.hash_salts); + auto hash_salts = FlatVector::GetData(state.hash_salts); for (idx_t r = 0; r < groups.size(); r++) { const auto &hash = hashes[r]; ht_offsets[r] = ApplyBitMask(hash); @@ -369,29 +369,20 @@ idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(DataChunk &groups, V for (idx_t i = 0; i < remaining_entries; i++) { const auto index = sel_vector->get_index(i); const auto &salt = hash_salts[index]; - auto &ht_offset = ht_offsets[index]; - while (true) { - auto &entry = entries[ht_offset]; - if (entry.IsOccupied()) { // Cell is occupied: Compare salts - if (entry.GetSalt() == salt) { - // Same salt, compare group keys - state.group_compare_vector.set_index(need_compare_count++, index); - break; - } else { - // Different salts, move to next entry (linear probing) - if (++ht_offset >= capacity) { - ht_offset = 0; - } - continue; - } - } else { // Cell is unoccupied, let's claim it - // Set salt (also marks as occupied) - entry.SetSalt(salt); - // Update selection lists for outer loops - state.empty_vector.set_index(new_entry_count++, index); - new_groups_out.set_index(new_group_count++, index); - break; + auto &entry = entries[ht_offsets[index]]; + if (entry.IsOccupied()) { // Cell is occupied: Compare salts + if (entry.GetSalt() == salt) { + state.group_compare_vector.set_index(need_compare_count++, index); + } else { + state.no_match_vector.set_index(no_match_count++, index); } + } else { // Cell is unoccupied + // Set salt (also marks as occupied) + entry.SetSalt(salt); + + // Update selection lists for outer loops + state.empty_vector.set_index(new_entry_count++, index); + new_groups_out.set_index(new_group_count++, index); } } @@ -431,10 +422,10 @@ idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(DataChunk &groups, V // Linear probing: each of the entries that do not match move to the next entry in the HT for (idx_t i = 0; i < no_match_count; i++) { - const auto index = state.no_match_vector.get_index(i); - auto &ht_offset = ht_offsets[index]; - if (++ht_offset >= capacity) { - ht_offset = 0; + idx_t index = state.no_match_vector.get_index(i); + ht_offsets[index]++; + if (ht_offsets[index] >= capacity) { + ht_offsets[index] = 0; } } sel_vector = &state.no_match_vector; diff --git a/src/duckdb/src/execution/expression_executor/execute_comparison.cpp b/src/duckdb/src/execution/expression_executor/execute_comparison.cpp index 7ca514620..85b18a287 100644 --- a/src/duckdb/src/execution/expression_executor/execute_comparison.cpp +++ b/src/duckdb/src/execution/expression_executor/execute_comparison.cpp @@ -95,6 +95,7 @@ static idx_t TemplatedSelectOperation(Vector &left, Vector &right, const Selecti return BinaryExecutor::Select(left, right, sel, count, true_sel, false_sel); case PhysicalType::LIST: case PhysicalType::STRUCT: + case PhysicalType::ARRAY: return NestedSelectOperation(left, right, sel, count, true_sel, false_sel); default: throw InternalException("Invalid type for comparison"); diff --git a/src/duckdb/src/execution/nested_loop_join/nested_loop_join_mark.cpp b/src/duckdb/src/execution/nested_loop_join/nested_loop_join_mark.cpp index 853037e50..67f047116 100644 --- a/src/duckdb/src/execution/nested_loop_join/nested_loop_join_mark.cpp +++ b/src/duckdb/src/execution/nested_loop_join/nested_loop_join_mark.cpp @@ -119,6 +119,7 @@ static void MarkJoinComparisonSwitch(Vector &left, Vector &right, idx_t lcount, switch (left.GetType().InternalType()) { case PhysicalType::STRUCT: case PhysicalType::LIST: + case PhysicalType::ARRAY: return MarkJoinNested(left, right, lcount, rcount, found_match, comparison_type); default: break; diff --git a/src/duckdb/src/execution/operator/csv_scanner/buffered_csv_reader.cpp b/src/duckdb/src/execution/operator/csv_scanner/buffered_csv_reader.cpp index 19e02da16..ca7265ee4 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/buffered_csv_reader.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/buffered_csv_reader.cpp @@ -18,6 +18,7 @@ #include "duckdb/storage/data_table.hpp" #include "utf8proc.hpp" #include "utf8proc_wrapper.hpp" +#include "duckdb/common/set.hpp" #include #include @@ -97,10 +98,11 @@ string BufferedCSVReader::ColumnTypesError(case_insensitive_map_t sql_typ return string(); } string exception = "COLUMN_TYPES error: Columns with names: "; + set problematic_columns; for (auto &col : sql_types_per_column) { - exception += "\"" + col.first + "\","; + problematic_columns.insert("\"" + col.first + "\""); } - exception.pop_back(); + exception += StringUtil::Join(problematic_columns, ","); exception += " do not exist in the CSV File"; return exception; } diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp index ceadfaa88..ed8e7bb56 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp @@ -348,7 +348,6 @@ void CSVReaderOptions::FromNamedParameters(named_parameter_map_t &in, ClientCont } auto loption = StringUtil::Lower(kv.first); if (loption == "columns") { - explicitly_set_columns = true; auto &child_type = kv.second.type(); if (child_type.id() != LogicalTypeId::STRUCT) { throw BinderException("read_csv columns requires a struct as input"); diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/csv_sniffer.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/csv_sniffer.cpp index 59b68baf8..37fd2c52b 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/sniffer/csv_sniffer.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/csv_sniffer.cpp @@ -3,9 +3,9 @@ namespace duckdb { CSVSniffer::CSVSniffer(CSVReaderOptions &options_p, shared_ptr buffer_manager_p, - CSVStateMachineCache &state_machine_cache_p, bool explicit_set_columns_p) + CSVStateMachineCache &state_machine_cache_p, SetColumns set_columns_p) : state_machine_cache(state_machine_cache_p), options(options_p), buffer_manager(std::move(buffer_manager_p)), - explicit_set_columns(explicit_set_columns_p) { + set_columns(set_columns_p) { // Check if any type is BLOB for (auto &type : options.sql_type_list) { @@ -20,14 +20,36 @@ CSVSniffer::CSVSniffer(CSVReaderOptions &options_p, shared_ptr auto &logical_type = format_template.first; best_format_candidates[logical_type].clear(); } + // Initialize max columns found to either 0 or however many were set + max_columns_found = set_columns.Size(); } +bool SetColumns::IsSet() { + if (!types) { + return false; + } + return !types->empty(); +} + +idx_t SetColumns::Size() { + if (!types) { + return 0; + } + return types->size(); +} + +// Set the CSV Options in the reference void CSVSniffer::SetResultOptions() { + bool og_header = options.dialect_options.header; options.dialect_options = best_candidate->dialect_options; options.dialect_options.new_line = best_candidate->dialect_options.new_line; - options.has_header = best_candidate->dialect_options.header; options.skip_rows_set = options.dialect_options.skip_rows > 0; if (options.has_header) { + // If header was manually set, we ignore the sniffer findings + options.dialect_options.header = og_header; + } + options.has_header = true; + if (options.dialect_options.header) { options.dialect_options.true_start = best_start_with_header; } else { options.dialect_options.true_start = best_start_without_header; @@ -43,7 +65,7 @@ SnifferResult CSVSniffer::SniffCSV() { RefineTypes(); // 4. Header Detection DetectHeader(); - if (explicit_set_columns) { + if (set_columns.IsSet()) { SetResultOptions(); // We do not need to run type refinement, since the types have been given by the user return SnifferResult({}, {}); diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/dialect_detection.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/dialect_detection.cpp index 85234f065..b3217ac33 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/sniffer/dialect_detection.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/dialect_detection.cpp @@ -56,6 +56,9 @@ struct SniffDialect { if (machine.cur_rows < STANDARD_VECTOR_SIZE && machine.state != CSVState::EMPTY_LINE) { sniffed_column_counts[machine.cur_rows++] = machine.column_count; } + if (machine.cur_rows == 0 && machine.state == CSVState::EMPTY_LINE) { + sniffed_column_counts[machine.cur_rows++] = machine.column_count; + } NewLineIdentifier suggested_newline; if (machine.carry_on_separator) { if (machine.single_record_separator) { @@ -78,6 +81,13 @@ struct SniffDialect { } }; +bool IsQuoteDefault(char quote) { + if (quote == '\"' || quote == '\'' || quote == '\0') { + return true; + } + return false; +} + void CSVSniffer::GenerateCandidateDetectionSearchSpace(vector &delim_candidates, vector "erule_candidates, unordered_map> "e_candidates_map, @@ -94,6 +104,11 @@ void CSVSniffer::GenerateCandidateDetectionSearchSpace(vector &delim_candi quote_candidates_map[(uint8_t)QuoteRule::QUOTES_RFC] = {options.dialect_options.state_machine_options.quote}; quote_candidates_map[(uint8_t)QuoteRule::QUOTES_OTHER] = {options.dialect_options.state_machine_options.quote}; quote_candidates_map[(uint8_t)QuoteRule::NO_QUOTES] = {options.dialect_options.state_machine_options.quote}; + // also add it as a escape rule + if (!IsQuoteDefault(options.dialect_options.state_machine_options.quote)) { + escape_candidates_map[(uint8_t)QuoteRule::QUOTES_RFC].emplace_back( + options.dialect_options.state_machine_options.quote); + } } else { // no quote rule provided: use standard/common quotes quote_candidates_map[(uint8_t)QuoteRule::QUOTES_RFC] = {'\"'}; @@ -151,10 +166,20 @@ void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machi if (sniffed_column_counts.size() > rows_read) { rows_read = sniffed_column_counts.size(); } + if (set_columns.IsCandidateUnacceptable(num_cols, options.null_padding, options.ignore_errors)) { + // Not acceptable + return; + } for (idx_t row = 0; row < sniffed_column_counts.size(); row++) { - if (sniffed_column_counts[row] == num_cols) { + if (set_columns.IsCandidateUnacceptable(sniffed_column_counts[row], options.null_padding, + options.ignore_errors)) { + // Not acceptable + return; + } + if (sniffed_column_counts[row] == num_cols || options.ignore_errors) { consistent_rows++; - } else if (num_cols < sniffed_column_counts[row] && !options.skip_rows_set) { + } else if (num_cols < sniffed_column_counts[row] && !options.skip_rows_set && + (!set_columns.IsSet() || options.null_padding)) { // all rows up to this point will need padding padding_count = 0; // we use the maximum amount of num_cols that we find @@ -209,6 +234,10 @@ void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machi (single_column_before || (more_values && !require_more_padding) || (more_than_one_column && require_less_padding)) && !invalid_padding) { + if (!candidates.empty() && set_columns.IsSet() && max_columns_found == candidates.size()) { + // We have a candidate that fits our requirements better + return; + } best_consistent_rows = consistent_rows; max_columns_found = num_cols; prev_padding_count = padding_count; @@ -241,11 +270,13 @@ void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machi bool CSVSniffer::RefineCandidateNextChunk(CSVStateMachine &candidate) { vector sniffed_column_counts(STANDARD_VECTOR_SIZE); candidate.csv_buffer_iterator.Process(candidate, sniffed_column_counts); - bool allow_padding = options.null_padding; - - for (idx_t row = 0; row < sniffed_column_counts.size(); row++) { - if (max_columns_found != sniffed_column_counts[row] && !allow_padding) { - return false; + for (auto &num_cols : sniffed_column_counts) { + if (set_columns.IsSet()) { + return !set_columns.IsCandidateUnacceptable(num_cols, options.null_padding, options.ignore_errors); + } else { + if (max_columns_found != num_cols && (!options.null_padding && !options.ignore_errors)) { + return false; + } } } return true; diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_detection.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_detection.cpp index c22eb9063..552cdaa5d 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_detection.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_detection.cpp @@ -332,6 +332,10 @@ void CSVSniffer::DetectTypes() { const idx_t start_idx_detection = row_idx; for (; row_idx < tuples.size(); row_idx++) { for (idx_t col = 0; col < tuples[row_idx].values.size(); col++) { + if (options.ignore_errors && col >= max_columns_found) { + // ignore this, since it's an error. + continue; + } auto &col_type_candidates = info_sql_types_candidates[col]; // col_type_candidates can't be empty since anything in a CSV file should at least be a string // and we validate utf-8 compatibility when creating the type diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_refinement.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_refinement.cpp index 8500f68ff..e602d0550 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_refinement.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_refinement.cpp @@ -22,6 +22,9 @@ struct Parse { // Started a new value // Check if it's UTF-8 (Or not?) machine.VerifyUTF8(); + if (machine.column_count >= parse_chunk.ColumnCount() && machine.options.ignore_errors) { + return false; + } auto &v = parse_chunk.data[machine.column_count++]; auto parse_data = FlatVector::GetData(v); if (machine.value.empty()) { @@ -34,7 +37,8 @@ struct Parse { } if (((machine.previous_state == CSVState::RECORD_SEPARATOR && machine.state != CSVState::EMPTY_LINE) || (machine.state != CSVState::RECORD_SEPARATOR && carriage_return)) && - machine.options.null_padding && machine.column_count < parse_chunk.ColumnCount()) { + (machine.options.null_padding || machine.options.ignore_errors) && + machine.column_count < parse_chunk.ColumnCount()) { // It's a new row, check if we need to pad stuff while (machine.column_count < parse_chunk.ColumnCount()) { auto &v = parse_chunk.data[machine.column_count++]; @@ -63,19 +67,22 @@ struct Parse { inline static void Finalize(CSVStateMachine &machine, DataChunk &parse_chunk) { if (machine.cur_rows < STANDARD_VECTOR_SIZE && machine.state != CSVState::EMPTY_LINE) { machine.VerifyUTF8(); - auto &v = parse_chunk.data[machine.column_count++]; - auto parse_data = FlatVector::GetData(v); - if (machine.value.empty()) { - auto &validity_mask = FlatVector::Validity(v); - validity_mask.SetInvalid(machine.cur_rows); - } else { - parse_data[machine.cur_rows] = StringVector::AddStringOrBlob(v, string_t(machine.value)); - } - while (machine.column_count < parse_chunk.ColumnCount()) { - auto &v_pad = parse_chunk.data[machine.column_count++]; - auto &validity_mask = FlatVector::Validity(v_pad); - validity_mask.SetInvalid(machine.cur_rows); + if (machine.column_count < parse_chunk.ColumnCount() || !machine.options.ignore_errors) { + auto &v = parse_chunk.data[machine.column_count++]; + auto parse_data = FlatVector::GetData(v); + if (machine.value.empty()) { + auto &validity_mask = FlatVector::Validity(v); + validity_mask.SetInvalid(machine.cur_rows); + } else { + parse_data[machine.cur_rows] = StringVector::AddStringOrBlob(v, string_t(machine.value)); + } + while (machine.column_count < parse_chunk.ColumnCount()) { + auto &v_pad = parse_chunk.data[machine.column_count++]; + auto &validity_mask = FlatVector::Validity(v_pad); + validity_mask.SetInvalid(machine.cur_rows); + } } + machine.cur_rows++; } parse_chunk.SetCardinality(machine.cur_rows); diff --git a/src/duckdb/src/execution/operator/join/physical_nested_loop_join.cpp b/src/duckdb/src/execution/operator/join/physical_nested_loop_join.cpp index 9036748e8..1d7927290 100644 --- a/src/duckdb/src/execution/operator/join/physical_nested_loop_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_nested_loop_join.cpp @@ -113,7 +113,8 @@ bool PhysicalNestedLoopJoin::IsSupported(const vector &conditions } for (auto &cond : conditions) { if (cond.left->return_type.InternalType() == PhysicalType::STRUCT || - cond.left->return_type.InternalType() == PhysicalType::LIST) { + cond.left->return_type.InternalType() == PhysicalType::LIST || + cond.left->return_type.InternalType() == PhysicalType::ARRAY) { return false; } } diff --git a/src/duckdb/src/execution/operator/persistent/physical_batch_copy_to_file.cpp b/src/duckdb/src/execution/operator/persistent/physical_batch_copy_to_file.cpp index 528577685..f951ddc44 100644 --- a/src/duckdb/src/execution/operator/persistent/physical_batch_copy_to_file.cpp +++ b/src/duckdb/src/execution/operator/persistent/physical_batch_copy_to_file.cpp @@ -175,10 +175,8 @@ void PhysicalBatchCopyToFile::FlushBatchData(ClientContext &context, GlobalSinkS //===--------------------------------------------------------------------===// // Next Batch //===--------------------------------------------------------------------===// -SinkNextBatchType PhysicalBatchCopyToFile::NextBatch(ExecutionContext &context, - OperatorSinkNextBatchInput &input) const { - auto &lstate = input.local_state; - auto &gstate_p = input.global_state; +void PhysicalBatchCopyToFile::NextBatch(ExecutionContext &context, GlobalSinkState &gstate_p, + LocalSinkState &lstate) const { auto &state = lstate.Cast(); if (state.collection && state.collection->Count() > 0) { // we finished processing this batch @@ -190,7 +188,6 @@ SinkNextBatchType PhysicalBatchCopyToFile::NextBatch(ExecutionContext &context, state.batch_index = lstate.partition_info.batch_index.GetIndex(); state.InitializeCollection(context.client, *this); - return SinkNextBatchType::READY; } unique_ptr PhysicalBatchCopyToFile::GetLocalSinkState(ExecutionContext &context) const { diff --git a/src/duckdb/src/execution/operator/persistent/physical_batch_insert.cpp b/src/duckdb/src/execution/operator/persistent/physical_batch_insert.cpp index d0da7d62f..d807e0e9f 100644 --- a/src/duckdb/src/execution/operator/persistent/physical_batch_insert.cpp +++ b/src/duckdb/src/execution/operator/persistent/physical_batch_insert.cpp @@ -298,9 +298,9 @@ unique_ptr PhysicalBatchInsert::GetLocalSinkState(ExecutionConte return make_uniq(context.client, insert_types, bound_defaults); } -SinkNextBatchType PhysicalBatchInsert::NextBatch(ExecutionContext &context, OperatorSinkNextBatchInput &input) const { - auto &gstate = input.global_state.Cast(); - auto &lstate = input.local_state.Cast(); +void PhysicalBatchInsert::NextBatch(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate_p) const { + auto &gstate = state.Cast(); + auto &lstate = lstate_p.Cast(); auto &table = gstate.table; auto batch_index = lstate.partition_info.batch_index.GetIndex(); @@ -316,7 +316,6 @@ SinkNextBatchType PhysicalBatchInsert::NextBatch(ExecutionContext &context, Oper lstate.CreateNewCollection(table, insert_types); } lstate.current_index = batch_index; - return SinkNextBatchType::READY; } SinkResultType PhysicalBatchInsert::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const { diff --git a/src/duckdb/src/execution/operator/persistent/physical_fixed_batch_copy.cpp b/src/duckdb/src/execution/operator/persistent/physical_fixed_batch_copy.cpp index 983ce31d1..27597c45c 100644 --- a/src/duckdb/src/execution/operator/persistent/physical_fixed_batch_copy.cpp +++ b/src/duckdb/src/execution/operator/persistent/physical_fixed_batch_copy.cpp @@ -450,10 +450,8 @@ void PhysicalFixedBatchCopy::ExecuteTasks(ClientContext &context, GlobalSinkStat //===--------------------------------------------------------------------===// // Next Batch //===--------------------------------------------------------------------===// -SinkNextBatchType PhysicalFixedBatchCopy::NextBatch(ExecutionContext &context, - OperatorSinkNextBatchInput &input) const { - auto &lstate = input.local_state; - auto &gstate_p = input.global_state; +void PhysicalFixedBatchCopy::NextBatch(ExecutionContext &context, GlobalSinkState &gstate_p, + LocalSinkState &lstate) const { auto &state = lstate.Cast(); if (state.collection && state.collection->Count() > 0) { // we finished processing this batch @@ -470,7 +468,6 @@ SinkNextBatchType PhysicalFixedBatchCopy::NextBatch(ExecutionContext &context, state.batch_index = lstate.partition_info.batch_index.GetIndex(); state.InitializeCollection(context.client, *this); - return SinkNextBatchType::READY; } unique_ptr PhysicalFixedBatchCopy::GetLocalSinkState(ExecutionContext &context) const { diff --git a/src/duckdb/src/execution/physical_operator.cpp b/src/duckdb/src/execution/physical_operator.cpp index accd42841..b97190a24 100644 --- a/src/duckdb/src/execution/physical_operator.cpp +++ b/src/duckdb/src/execution/physical_operator.cpp @@ -106,8 +106,7 @@ SinkFinalizeType PhysicalOperator::Finalize(Pipeline &pipeline, Event &event, Cl return SinkFinalizeType::READY; } -SinkNextBatchType PhysicalOperator::NextBatch(ExecutionContext &context, OperatorSinkNextBatchInput &input) const { - return SinkNextBatchType::READY; +void PhysicalOperator::NextBatch(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate_p) const { } unique_ptr PhysicalOperator::GetLocalSinkState(ExecutionContext &context) const { @@ -219,6 +218,7 @@ bool CachingPhysicalOperator::CanCacheType(const LogicalType &type) { switch (type.id()) { case LogicalTypeId::LIST: case LogicalTypeId::MAP: + case LogicalTypeId::ARRAY: return false; case LogicalTypeId::STRUCT: { auto &entries = StructType::GetChildTypes(type); diff --git a/src/duckdb/src/execution/physical_plan/plan_comparison_join.cpp b/src/duckdb/src/execution/physical_plan/plan_comparison_join.cpp index f03b49150..2dcd035fb 100644 --- a/src/duckdb/src/execution/physical_plan/plan_comparison_join.cpp +++ b/src/duckdb/src/execution/physical_plan/plan_comparison_join.cpp @@ -75,6 +75,7 @@ void CheckForPerfectJoinOpt(LogicalComparisonJoin &op, PerfectHashJoinStats &joi switch (type.InternalType()) { case PhysicalType::STRUCT: case PhysicalType::LIST: + case PhysicalType::ARRAY: return; default: break; diff --git a/src/duckdb/src/execution/physical_plan/plan_window.cpp b/src/duckdb/src/execution/physical_plan/plan_window.cpp index 5e2e502c1..34556115b 100644 --- a/src/duckdb/src/execution/physical_plan/plan_window.cpp +++ b/src/duckdb/src/execution/physical_plan/plan_window.cpp @@ -12,7 +12,8 @@ namespace duckdb { static bool IsStreamingWindow(unique_ptr &expr) { auto &wexpr = expr->Cast(); - if (!wexpr.partitions.empty() || !wexpr.orders.empty() || wexpr.ignore_nulls) { + if (!wexpr.partitions.empty() || !wexpr.orders.empty() || wexpr.ignore_nulls || + wexpr.exclude_clause != WindowExcludeMode::NO_OTHER) { return false; } switch (wexpr.type) { diff --git a/src/duckdb/src/execution/radix_partitioned_hashtable.cpp b/src/duckdb/src/execution/radix_partitioned_hashtable.cpp index 9a1069609..30b6959ee 100644 --- a/src/duckdb/src/execution/radix_partitioned_hashtable.cpp +++ b/src/duckdb/src/execution/radix_partitioned_hashtable.cpp @@ -347,7 +347,7 @@ bool MaybeRepartition(ClientContext &context, RadixHTGlobalSinkState &gstate, Ra } // We can go external when there is only one active thread, but we shouldn't repartition here - if (gstate.active_threads < 2) { + if (gstate.count_before_combining < 2) { return false; } diff --git a/src/duckdb/src/execution/window_executor.cpp b/src/duckdb/src/execution/window_executor.cpp index 91d39183c..53841b727 100644 --- a/src/duckdb/src/execution/window_executor.cpp +++ b/src/duckdb/src/execution/window_executor.cpp @@ -59,7 +59,8 @@ static idx_t FindPrevStart(const ValidityMask &mask, const idx_t l, idx_t r, idx // Loop backwards over the block // shift is probing r-1 >= l >= 0 - for (++shift; shift-- > 0; --r) { + for (++shift; shift-- > 0 && l < r; --r) { + // l < r ensures n == 1 if result is supposed to be NULL because of EXCLUDE if (mask.RowIsValid(block, shift) && --n == 0) { return MaxValue(l, r - 1); } @@ -544,7 +545,8 @@ WindowBoundariesState::WindowBoundariesState(BoundWindowExpression &wexpr, const partition_count(wexpr.partitions.size()), order_count(wexpr.orders.size()), range_sense(wexpr.orders.empty() ? OrderType::INVALID : wexpr.orders[0].type), has_preceding_range(HasPrecedingRange(wexpr)), has_following_range(HasFollowingRange(wexpr)), - needs_peer(BoundaryNeedsPeer(wexpr.end) || ExpressionNeedsPeer(wexpr.type)) { + needs_peer(BoundaryNeedsPeer(wexpr.end) || ExpressionNeedsPeer(wexpr.type) || + wexpr.exclude_clause >= WindowExcludeMode::GROUP) { } void WindowBoundariesState::Bounds(DataChunk &bounds, idx_t row_idx, const WindowInputColumn &range, const idx_t count, @@ -614,6 +616,134 @@ void WindowExecutorBoundsState::UpdateBounds(idx_t row_idx, DataChunk &input_chu state.Bounds(bounds, row_idx, range, count, boundary_start, boundary_end, partition_mask, order_mask); } +//===--------------------------------------------------------------------===// +// ExclusionFilter +//===--------------------------------------------------------------------===// + +//! Handles window exclusion by piggybacking on the filtering logic. +//! (needed for first_value, last_value, nth_value) +class ExclusionFilter { +public: + ExclusionFilter(const WindowExcludeMode exclude_mode_p, idx_t total_count, const ValidityMask &src) + : mode(exclude_mode_p), mask_src(src) { + mask.Initialize(total_count); + + // copy the data from mask_src + FetchFromSource(0, total_count); + } + + //! Copy the entries from mask_src to mask, in the index range [begin, end) + void FetchFromSource(idx_t begin, idx_t end); + //! Apply the current exclusion to the validity mask + //! (offset is the current row's index within the chunk) + void ApplyExclusion(DataChunk &bounds, idx_t row_idx, idx_t offset); + //! Reset the validity mask to match mask_src + //! (offset is the current row's index within the chunk) + void ResetMask(idx_t row_idx, idx_t offset); + + //! The current peer group's begin + idx_t curr_peer_begin; + //! The current peer group's end + idx_t curr_peer_end; + //! The window exclusion mode + WindowExcludeMode mode; + //! The validity mask representing the exclusion + ValidityMask mask; + //! The validity mask upon which mask is based + const ValidityMask &mask_src; + //! A validity mask consisting of only one entries (needed if no ignore_nulls mask is supplied) + ValidityMask all_ones_mask; +}; + +void ExclusionFilter::FetchFromSource(idx_t begin, idx_t end) { + idx_t begin_entry_idx; + idx_t end_entry_idx; + idx_t idx_in_entry; + mask.GetEntryIndex(begin, begin_entry_idx, idx_in_entry); + mask.GetEntryIndex(end - 1, end_entry_idx, idx_in_entry); + auto dst = mask.GetData() + begin_entry_idx; + for (idx_t entry_idx = begin_entry_idx; entry_idx <= end_entry_idx; ++entry_idx) { + *dst++ = mask_src.GetValidityEntry(entry_idx); + } +} + +void ExclusionFilter::ApplyExclusion(DataChunk &bounds, idx_t row_idx, idx_t offset) { + // flip the bits in mask according to the window exclusion mode + switch (mode) { + case WindowExcludeMode::CURRENT_ROW: + mask.SetInvalid(row_idx); + break; + case WindowExcludeMode::TIES: + case WindowExcludeMode::GROUP: { + if (curr_peer_end == row_idx || offset == 0) { + // new peer group or input chunk: set entire peer group to invalid + auto peer_begin = FlatVector::GetData(bounds.data[PEER_BEGIN]); + auto peer_end = FlatVector::GetData(bounds.data[PEER_END]); + curr_peer_begin = peer_begin[offset]; + curr_peer_end = peer_end[offset]; + for (idx_t i = curr_peer_begin; i < curr_peer_end; i++) { + mask.SetInvalid(i); + } + } + if (mode == WindowExcludeMode::TIES) { + mask.Set(row_idx, mask_src.RowIsValid(row_idx)); + } + break; + } + default: + break; + } +} + +void ExclusionFilter::ResetMask(idx_t row_idx, idx_t offset) { + // flip the bits that were modified in ApplyExclusion back + switch (mode) { + case WindowExcludeMode::CURRENT_ROW: + mask.Set(row_idx, mask_src.RowIsValid(row_idx)); + break; + case WindowExcludeMode::TIES: + mask.SetInvalid(row_idx); + DUCKDB_EXPLICIT_FALLTHROUGH; + case WindowExcludeMode::GROUP: + if (curr_peer_end == row_idx + 1) { + // if we've reached the peer group's end, restore the entire peer group + FetchFromSource(curr_peer_begin, curr_peer_end); + } + break; + default: + break; + } +} + +//===--------------------------------------------------------------------===// +// WindowValueState +//===--------------------------------------------------------------------===// + +//! A class representing the state of the first_value, last_value and nth_value functions +class WindowValueState : public WindowExecutorBoundsState { +public: + WindowValueState(BoundWindowExpression &wexpr, ClientContext &context, const idx_t count, + const ValidityMask &partition_mask_p, const ValidityMask &order_mask_p, + const ValidityMask &ignore_nulls) + : WindowExecutorBoundsState(wexpr, context, count, partition_mask_p, order_mask_p) + + { + if (wexpr.exclude_clause == WindowExcludeMode::NO_OTHER) { + exclusion_filter = nullptr; + ignore_nulls_exclude = &ignore_nulls; + } else { + // create the exclusion filter based on ignore_nulls + exclusion_filter = make_uniq(wexpr.exclude_clause, count, ignore_nulls); + ignore_nulls_exclude = &exclusion_filter->mask; + } + } + + //! The exclusion filter handling exclusion + unique_ptr exclusion_filter; + //! The validity mask that combines both the NULLs and exclusion information + const ValidityMask *ignore_nulls_exclude; +}; + //===--------------------------------------------------------------------===// // WindowExecutor //===--------------------------------------------------------------------===// @@ -663,6 +793,10 @@ bool WindowAggregateExecutor::IsConstantAggregate() { if (!wexpr.aggregate) { return false; } + // window exclusion cannot be handled by constant aggregates + if (wexpr.exclude_clause != WindowExcludeMode::NO_OTHER) { + return false; + } // COUNT(*) is already handled efficiently by segment trees. if (wexpr.children.empty()) { @@ -744,14 +878,16 @@ WindowAggregateExecutor::WindowAggregateExecutor(BoundWindowExpression &wexpr, C // Check for constant aggregate if (IsConstantAggregate()) { - aggregator = - make_uniq(AggregateObject(wexpr), wexpr.return_type, partition_mask, count); + aggregator = make_uniq(AggregateObject(wexpr), wexpr.return_type, partition_mask, + wexpr.exclude_clause, count); } else if (IsCustomAggregate()) { - aggregator = make_uniq(AggregateObject(wexpr), wexpr.return_type, count); + aggregator = + make_uniq(AggregateObject(wexpr), wexpr.return_type, wexpr.exclude_clause, count); } else if (wexpr.aggregate) { // build a segment tree for frame-adhering aggregates // see http://www.vldb.org/pvldb/vol8/p1058-leis.pdf - aggregator = make_uniq(AggregateObject(wexpr), wexpr.return_type, count, mode); + aggregator = + make_uniq(AggregateObject(wexpr), wexpr.return_type, mode, wexpr.exclude_clause, count); } // evaluate the FILTER clause and stuff it into a large mask for compactness and reuse @@ -799,22 +935,25 @@ class WindowAggregateState : public WindowExecutorBoundsState { } public: + // state of aggregator unique_ptr aggregator_state; void NextRank(idx_t partition_begin, idx_t peer_begin, idx_t row_idx); }; unique_ptr WindowAggregateExecutor::GetExecutorState() const { - return make_uniq(wexpr, context, payload_count, partition_mask, order_mask, *aggregator); + auto res = make_uniq(wexpr, context, payload_count, partition_mask, order_mask, *aggregator); + return std::move(res); } void WindowAggregateExecutor::EvaluateInternal(WindowExecutorState &lstate, Vector &result, idx_t count, idx_t row_idx) const { auto &lastate = lstate.Cast(); D_ASSERT(aggregator); - auto window_begin = FlatVector::GetData(lastate.bounds.data[WINDOW_BEGIN]); - auto window_end = FlatVector::GetData(lastate.bounds.data[WINDOW_END]); - aggregator->Evaluate(*lastate.aggregator_state, window_begin, window_end, result, count); + + auto &agg_state = *lastate.aggregator_state; + + aggregator->Evaluate(agg_state, lastate.bounds, result, count, row_idx); } //===--------------------------------------------------------------------===// @@ -1081,6 +1220,15 @@ void WindowValueExecutor::Sink(DataChunk &input_chunk, const idx_t input_idx, co WindowExecutor::Sink(input_chunk, input_idx, total_count); } +unique_ptr WindowValueExecutor::GetExecutorState() const { + if (wexpr.type == ExpressionType::WINDOW_FIRST_VALUE || wexpr.type == ExpressionType::WINDOW_LAST_VALUE || + wexpr.type == ExpressionType::WINDOW_NTH_VALUE) { + return make_uniq(wexpr, context, payload_count, partition_mask, order_mask, ignore_nulls); + } else { + return make_uniq(wexpr, context, payload_count, partition_mask, order_mask); + } +} + void WindowNtileExecutor::EvaluateInternal(WindowExecutorState &lstate, Vector &result, idx_t count, idx_t row_idx) const { D_ASSERT(payload_collection.ColumnCount() == 1); @@ -1210,22 +1358,31 @@ WindowFirstValueExecutor::WindowFirstValueExecutor(BoundWindowExpression &wexpr, void WindowFirstValueExecutor::EvaluateInternal(WindowExecutorState &lstate, Vector &result, idx_t count, idx_t row_idx) const { - auto &lbstate = lstate.Cast(); - auto window_begin = FlatVector::GetData(lbstate.bounds.data[WINDOW_BEGIN]); - auto window_end = FlatVector::GetData(lbstate.bounds.data[WINDOW_END]); + auto &lvstate = lstate.Cast(); + auto window_begin = FlatVector::GetData(lvstate.bounds.data[WINDOW_BEGIN]); + auto window_end = FlatVector::GetData(lvstate.bounds.data[WINDOW_END]); for (idx_t i = 0; i < count; ++i, ++row_idx) { + + if (lvstate.exclusion_filter) { + lvstate.exclusion_filter->ApplyExclusion(lvstate.bounds, row_idx, i); + } + if (window_begin[i] >= window_end[i]) { FlatVector::SetNull(result, i, true); continue; } // Same as NTH_VALUE(..., 1) idx_t n = 1; - const auto first_idx = FindNextStart(ignore_nulls, window_begin[i], window_end[i], n); + const auto first_idx = FindNextStart(*lvstate.ignore_nulls_exclude, window_begin[i], window_end[i], n); if (!n) { CopyCell(payload_collection, 0, first_idx, result, i); } else { FlatVector::SetNull(result, i, true); } + + if (lvstate.exclusion_filter) { + lvstate.exclusion_filter->ResetMask(row_idx, i); + } } } @@ -1237,21 +1394,30 @@ WindowLastValueExecutor::WindowLastValueExecutor(BoundWindowExpression &wexpr, C void WindowLastValueExecutor::EvaluateInternal(WindowExecutorState &lstate, Vector &result, idx_t count, idx_t row_idx) const { - auto &lbstate = lstate.Cast(); - auto window_begin = FlatVector::GetData(lbstate.bounds.data[WINDOW_BEGIN]); - auto window_end = FlatVector::GetData(lbstate.bounds.data[WINDOW_END]); + auto &lvstate = lstate.Cast(); + auto window_begin = FlatVector::GetData(lvstate.bounds.data[WINDOW_BEGIN]); + auto window_end = FlatVector::GetData(lvstate.bounds.data[WINDOW_END]); for (idx_t i = 0; i < count; ++i, ++row_idx) { + + if (lvstate.exclusion_filter) { + lvstate.exclusion_filter->ApplyExclusion(lvstate.bounds, row_idx, i); + } + if (window_begin[i] >= window_end[i]) { FlatVector::SetNull(result, i, true); continue; } idx_t n = 1; - const auto last_idx = FindPrevStart(ignore_nulls, window_begin[i], window_end[i], n); + const auto last_idx = FindPrevStart(*lvstate.ignore_nulls_exclude, window_begin[i], window_end[i], n); if (!n) { CopyCell(payload_collection, 0, last_idx, result, i); } else { FlatVector::SetNull(result, i, true); } + + if (lvstate.exclusion_filter) { + lvstate.exclusion_filter->ResetMask(row_idx, i); + } } } @@ -1265,10 +1431,15 @@ void WindowNthValueExecutor::EvaluateInternal(WindowExecutorState &lstate, Vecto idx_t row_idx) const { D_ASSERT(payload_collection.ColumnCount() == 2); - auto &lbstate = lstate.Cast(); - auto window_begin = FlatVector::GetData(lbstate.bounds.data[WINDOW_BEGIN]); - auto window_end = FlatVector::GetData(lbstate.bounds.data[WINDOW_END]); + auto &lvstate = lstate.Cast(); + auto window_begin = FlatVector::GetData(lvstate.bounds.data[WINDOW_BEGIN]); + auto window_end = FlatVector::GetData(lvstate.bounds.data[WINDOW_END]); for (idx_t i = 0; i < count; ++i, ++row_idx) { + + if (lvstate.exclusion_filter) { + lvstate.exclusion_filter->ApplyExclusion(lvstate.bounds, row_idx, i); + } + if (window_begin[i] >= window_end[i]) { FlatVector::SetNull(result, i, true); continue; @@ -1283,7 +1454,7 @@ void WindowNthValueExecutor::EvaluateInternal(WindowExecutorState &lstate, Vecto FlatVector::SetNull(result, i, true); } else { auto n = idx_t(n_param); - const auto nth_index = FindNextStart(ignore_nulls, window_begin[i], window_end[i], n); + const auto nth_index = FindNextStart(*lvstate.ignore_nulls_exclude, window_begin[i], window_end[i], n); if (!n) { CopyCell(payload_collection, 0, nth_index, result, i); } else { @@ -1291,6 +1462,10 @@ void WindowNthValueExecutor::EvaluateInternal(WindowExecutorState &lstate, Vecto } } } + + if (lvstate.exclusion_filter) { + lvstate.exclusion_filter->ResetMask(row_idx, i); + } } } diff --git a/src/duckdb/src/execution/window_segment_tree.cpp b/src/duckdb/src/execution/window_segment_tree.cpp index 2a66399c9..b783e4ef0 100644 --- a/src/duckdb/src/execution/window_segment_tree.cpp +++ b/src/duckdb/src/execution/window_segment_tree.cpp @@ -3,6 +3,7 @@ #include "duckdb/common/algorithm.hpp" #include "duckdb/common/helper.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/execution/window_executor.hpp" #include @@ -14,9 +15,10 @@ namespace duckdb { WindowAggregatorState::WindowAggregatorState() : allocator(Allocator::DefaultAllocator()) { } -WindowAggregator::WindowAggregator(AggregateObject aggr, const LogicalType &result_type_p, idx_t partition_count_p) +WindowAggregator::WindowAggregator(AggregateObject aggr, const LogicalType &result_type_p, + const WindowExcludeMode exclude_mode_p, idx_t partition_count_p) : aggr(std::move(aggr)), result_type(result_type_p), partition_count(partition_count_p), - state_size(aggr.function.state_size()), filter_pos(0) { + state_size(aggr.function.state_size()), filter_pos(0), exclude_mode(exclude_mode_p) { } WindowAggregator::~WindowAggregator() { @@ -50,8 +52,9 @@ void WindowAggregator::Finalize() { // WindowConstantAggregate //===--------------------------------------------------------------------===// WindowConstantAggregator::WindowConstantAggregator(AggregateObject aggr, const LogicalType &result_type, - const ValidityMask &partition_mask, const idx_t count) - : WindowAggregator(std::move(aggr), result_type, count), partition(0), row(0), state(state_size), + const ValidityMask &partition_mask, + const WindowExcludeMode exclude_mode_p, const idx_t count) + : WindowAggregator(std::move(aggr), result_type, exclude_mode_p, count), partition(0), row(0), state(state_size), statep(Value::POINTER(CastPointerToValue(state.data()))), statef(Value::POINTER(CastPointerToValue(state.data()))) { @@ -203,8 +206,9 @@ unique_ptr WindowConstantAggregator::GetLocalState() cons return make_uniq(); } -void WindowConstantAggregator::Evaluate(WindowAggregatorState &lstate, const idx_t *begins, const idx_t *ends, - Vector &target, idx_t count) const { +void WindowConstantAggregator::Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &target, + idx_t count, idx_t row_idx) const { + auto begins = FlatVector::GetData(bounds.data[WINDOW_BEGIN]); // Chunk up the constants and copy them one at a time auto &lcstate = lstate.Cast(); idx_t matched = 0; @@ -234,8 +238,9 @@ void WindowConstantAggregator::Evaluate(WindowAggregatorState &lstate, const idx //===--------------------------------------------------------------------===// // WindowCustomAggregator //===--------------------------------------------------------------------===// -WindowCustomAggregator::WindowCustomAggregator(AggregateObject aggr, const LogicalType &result_type, idx_t count) - : WindowAggregator(std::move(aggr), result_type, count) { +WindowCustomAggregator::WindowCustomAggregator(AggregateObject aggr, const LogicalType &result_type, + const WindowExcludeMode exclude_mode_p, idx_t count) + : WindowAggregator(std::move(aggr), result_type, exclude_mode_p, count) { } WindowCustomAggregator::~WindowCustomAggregator() { @@ -243,7 +248,7 @@ WindowCustomAggregator::~WindowCustomAggregator() { class WindowCustomAggregatorState : public WindowAggregatorState { public: - explicit WindowCustomAggregatorState(const AggregateObject &aggr, DataChunk &inputs); + WindowCustomAggregatorState(const AggregateObject &aggr, DataChunk &inputs, const WindowExcludeMode exclude_mode); ~WindowCustomAggregatorState() override; public: @@ -256,14 +261,30 @@ class WindowCustomAggregatorState : public WindowAggregatorState { //! Reused result state container for the window functions Vector statef; //! The frame boundaries, used for the window functions - FrameBounds frame; + vector frames; }; -WindowCustomAggregatorState::WindowCustomAggregatorState(const AggregateObject &aggr, DataChunk &inputs) +WindowCustomAggregatorState::WindowCustomAggregatorState(const AggregateObject &aggr, DataChunk &inputs, + const WindowExcludeMode exclude_mode) : aggr(aggr), inputs(inputs), state(aggr.function.state_size()), - statef(Value::POINTER(CastPointerToValue(state.data()))), frame(0, 0) { + statef(Value::POINTER(CastPointerToValue(state.data()))), frames(3, {0, 0}) { // if we have a frame-by-frame method, share the single state aggr.function.initialize(state.data()); + + idx_t nframes = 0; + switch (exclude_mode) { + case WindowExcludeMode::NO_OTHER: + nframes = 1; + break; + case WindowExcludeMode::TIES: + nframes = 3; + break; + case WindowExcludeMode::CURRENT_ROW: + case WindowExcludeMode::GROUP: + nframes = 2; + break; + } + frames.resize(nframes, {0, 0}); } WindowCustomAggregatorState::~WindowCustomAggregatorState() { @@ -274,41 +295,90 @@ WindowCustomAggregatorState::~WindowCustomAggregatorState() { } unique_ptr WindowCustomAggregator::GetLocalState() const { - return make_uniq(aggr, const_cast(inputs)); + return make_uniq(aggr, const_cast(inputs), exclude_mode); } -void WindowCustomAggregator::Evaluate(WindowAggregatorState &lstate, const idx_t *begins, const idx_t *ends, - Vector &result, idx_t count) const { +void WindowCustomAggregator::Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, + idx_t count, idx_t row_idx) const { + auto begins = FlatVector::GetData(bounds.data[WINDOW_BEGIN]); + auto ends = FlatVector::GetData(bounds.data[WINDOW_END]); + auto peer_begin = FlatVector::GetData(bounds.data[PEER_BEGIN]); + auto peer_end = FlatVector::GetData(bounds.data[PEER_END]); + // TODO: window should take a const Vector* auto &lcstate = lstate.Cast(); - auto &frame = lcstate.frame; + auto &frames = lcstate.frames; auto params = lcstate.inputs.data.data(); auto &rmask = FlatVector::Validity(result); - for (idx_t i = 0; i < count; ++i) { - const auto begin = begins[i]; - const auto end = ends[i]; - if (begin >= end) { + for (idx_t i = 0, cur_row = row_idx; i < count; ++i, ++cur_row) { + idx_t nframes = 0; + idx_t non_empty = 0; + if (exclude_mode == WindowExcludeMode::NO_OTHER) { + auto begin = begins[i]; + auto end = ends[i]; + non_empty += (begin < end); + frames[nframes++] = FrameBounds(begin, end); + } else { + // The frame_exclusion option allows rows around the current row to be excluded from the frame, + // even if they would be included according to the frame start and frame end options. + // EXCLUDE CURRENT ROW excludes the current row from the frame. + // EXCLUDE GROUP excludes the current row and its ordering peers from the frame. + // EXCLUDE TIES excludes any peers of the current row from the frame, but not the current row itself. + // EXCLUDE NO OTHERS simply specifies explicitly the default behavior + // of not excluding the current row or its peers. + // https://www.postgresql.org/docs/current/sql-expressions.html#SYNTAX-WINDOW-FUNCTIONS + // + // For the sake of the client, we make some guarantees about the subframes: + // * They are in order left-to-right + // * They do not intersect + // * start <= end + // * The number is always the same + // + // Since we always have peer_begin <= cur_row < cur_row + 1 <= peer_end + // this is not too hard to arrange, but it may be that some subframes are contiguous, + // and some are empty. + + // WindowSegmentTreePart::LEFT + auto begin = begins[i]; + auto end = (exclude_mode == WindowExcludeMode::CURRENT_ROW) ? cur_row : peer_begin[i]; + end = MaxValue(begin, end); + non_empty += (begin < end); + frames[nframes++] = FrameBounds(begin, end); + + // with EXCLUDE TIES, in addition to the frame part right of the peer group's end, + // we also need to consider the current row + if (exclude_mode == WindowExcludeMode::TIES) { + ++non_empty; + frames[nframes++] = FrameBounds(cur_row, cur_row + 1); + } + + // WindowSegmentTreePart::RIGHT + end = ends[i]; + begin = (exclude_mode == WindowExcludeMode::CURRENT_ROW) ? (cur_row + 1) : peer_end[i]; + begin = MinValue(begin, end); + non_empty += (begin < end); + frames[nframes++] = FrameBounds(begin, end); + } + + // No data means NULL + if (!non_empty) { rmask.SetInvalid(i); continue; } - // Frame boundaries - auto prev = frame; - frame = FrameBounds(begin, end); - // Extract the range AggregateInputData aggr_input_data(aggr.GetFunctionData(), lstate.allocator); - aggr.function.window(params, filter_mask, aggr_input_data, inputs.ColumnCount(), lcstate.state.data(), frame, - prev, result, i, 0); + aggr.function.window(params, filter_mask, aggr_input_data, inputs.ColumnCount(), lcstate.state.data(), frames, + result, i); } } //===--------------------------------------------------------------------===// // WindowSegmentTree //===--------------------------------------------------------------------===// -WindowSegmentTree::WindowSegmentTree(AggregateObject aggr, const LogicalType &result_type, idx_t count, - WindowAggregationMode mode_p) - : WindowAggregator(std::move(aggr), result_type, count), internal_nodes(0), mode(mode_p) { +WindowSegmentTree::WindowSegmentTree(AggregateObject aggr, const LogicalType &result_type, WindowAggregationMode mode_p, + const WindowExcludeMode exclude_mode_p, idx_t count) + : WindowAggregator(std::move(aggr), result_type, exclude_mode_p, count), internal_nodes(0), mode(mode_p) { } void WindowSegmentTree::Finalize() { @@ -342,18 +412,33 @@ WindowSegmentTree::~WindowSegmentTree() { } } -class WindowSegmentTreeState : public WindowAggregatorState { +class WindowSegmentTreePart { public: - WindowSegmentTreeState(const AggregateObject &aggr, DataChunk &inputs, const ValidityMask &filter_mask); - ~WindowSegmentTreeState() override; + enum FramePart : uint8_t { FULL = 0, LEFT = 1, RIGHT = 2 }; + + WindowSegmentTreePart(ArenaAllocator &allocator, const AggregateObject &aggr, DataChunk &inputs, + const ValidityMask &filter_mask); + ~WindowSegmentTreePart(); + + unique_ptr Copy() const { + return make_uniq(allocator, aggr, inputs, filter_mask); + } void FlushStates(bool combining); void ExtractFrame(idx_t begin, idx_t end, data_ptr_t current_state); void WindowSegmentValue(const WindowSegmentTree &tree, idx_t l_idx, idx_t begin, idx_t end, data_ptr_t current_state); + //! optionally writes result and calls destructors void Finalize(Vector &result, idx_t count); + void Combine(WindowSegmentTreePart &other, idx_t count); + + void Evaluate(const WindowSegmentTree &tree, const idx_t *begins, const idx_t *ends, Vector &result, idx_t count, + idx_t row_idx, FramePart frame_part); + public: + //! Allocator for aggregates + ArenaAllocator &allocator; //! The aggregate function const AggregateObject &aggr; //! The aggregate function @@ -378,11 +463,29 @@ class WindowSegmentTreeState : public WindowAggregatorState { idx_t flush_count; }; -WindowSegmentTreeState::WindowSegmentTreeState(const AggregateObject &aggr, DataChunk &inputs, - const ValidityMask &filter_mask) - : aggr(aggr), inputs(inputs), filter_mask(filter_mask), state_size(aggr.function.state_size()), - state(state_size * STANDARD_VECTOR_SIZE), statep(LogicalType::POINTER), statel(LogicalType::POINTER), - statef(LogicalType::POINTER), flush_count(0) { +class WindowSegmentTreeState : public WindowAggregatorState { +public: + WindowSegmentTreeState(const AggregateObject &aggr, DataChunk &inputs, const ValidityMask &filter_mask) + : aggr(aggr), inputs(inputs), filter_mask(filter_mask), part(allocator, aggr, inputs, filter_mask) { + } + + //! The aggregate function + const AggregateObject &aggr; + //! The aggregate function + DataChunk &inputs; + //! The filtered rows in inputs + const ValidityMask &filter_mask; + //! The left (default) segment tree part + WindowSegmentTreePart part; + //! The right segment tree part (for EXCLUDE) + unique_ptr right_part; +}; + +WindowSegmentTreePart::WindowSegmentTreePart(ArenaAllocator &allocator, const AggregateObject &aggr, DataChunk &inputs, + const ValidityMask &filter_mask) + : allocator(allocator), aggr(aggr), inputs(inputs), filter_mask(filter_mask), + state_size(aggr.function.state_size()), state(state_size * STANDARD_VECTOR_SIZE), statep(LogicalType::POINTER), + statel(LogicalType::POINTER), statef(LogicalType::POINTER), flush_count(0) { if (inputs.ColumnCount() > 0) { leaves.Initialize(Allocator::DefaultAllocator(), inputs.GetTypes()); filter_sel.Initialize(); @@ -400,14 +503,14 @@ WindowSegmentTreeState::WindowSegmentTreeState(const AggregateObject &aggr, Data } } -WindowSegmentTreeState::~WindowSegmentTreeState() { +WindowSegmentTreePart::~WindowSegmentTreePart() { } unique_ptr WindowSegmentTree::GetLocalState() const { return make_uniq(aggr, const_cast(inputs), filter_mask); } -void WindowSegmentTreeState::FlushStates(bool combining) { +void WindowSegmentTreePart::FlushStates(bool combining) { if (!flush_count) { return; } @@ -425,7 +528,12 @@ void WindowSegmentTreeState::FlushStates(bool combining) { flush_count = 0; } -void WindowSegmentTreeState::ExtractFrame(idx_t begin, idx_t end, data_ptr_t state_ptr) { +void WindowSegmentTreePart::Combine(WindowSegmentTreePart &other, idx_t count) { + AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); + aggr.function.combine(other.statef, statef, aggr_input_data, count); +} + +void WindowSegmentTreePart::ExtractFrame(idx_t begin, idx_t end, data_ptr_t state_ptr) { const auto count = end - begin; // If we are not filtering, @@ -453,8 +561,8 @@ void WindowSegmentTreeState::ExtractFrame(idx_t begin, idx_t end, data_ptr_t sta } } -void WindowSegmentTreeState::WindowSegmentValue(const WindowSegmentTree &tree, idx_t l_idx, idx_t begin, idx_t end, - data_ptr_t state_ptr) { +void WindowSegmentTreePart::WindowSegmentValue(const WindowSegmentTree &tree, idx_t l_idx, idx_t begin, idx_t end, + data_ptr_t state_ptr) { D_ASSERT(begin <= end); if (begin == end || inputs.ColumnCount() == 0) { return; @@ -479,8 +587,8 @@ void WindowSegmentTreeState::WindowSegmentValue(const WindowSegmentTree &tree, i } } } -void WindowSegmentTreeState::Finalize(Vector &result, idx_t count) { - // Finalise the result aggregates +void WindowSegmentTreePart::Finalize(Vector &result, idx_t count) { + // Finalise the result aggregates and write to result if write_result is set AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); aggr.function.finalize(statef, aggr_input_data, result, count, 0); @@ -494,7 +602,7 @@ void WindowSegmentTree::ConstructTree() { D_ASSERT(inputs.ColumnCount() > 0); // Use a temporary scan state to build the tree - auto >state = gstate->Cast(); + auto >state = gstate->Cast().part; // compute space required to store internal nodes of segment tree internal_nodes = 0; @@ -533,21 +641,71 @@ void WindowSegmentTree::ConstructTree() { } } -void WindowSegmentTree::Evaluate(WindowAggregatorState &lstate, const idx_t *begins, const idx_t *ends, Vector &result, - idx_t count) const { +void WindowSegmentTree::Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, idx_t count, + idx_t row_idx) const { + auto <state = lstate.Cast(); - const auto cant_combine = (!aggr.function.combine || !UseCombineAPI()); - auto fdata = FlatVector::GetData(ltstate.statef); + auto window_begin = FlatVector::GetData(bounds.data[WINDOW_BEGIN]); + auto window_end = FlatVector::GetData(bounds.data[WINDOW_END]); + auto peer_begin = FlatVector::GetData(bounds.data[PEER_BEGIN]); + auto peer_end = FlatVector::GetData(bounds.data[PEER_END]); + + auto &part = ltstate.part; + if (exclude_mode != WindowExcludeMode::NO_OTHER) { + // 1. evaluate the tree left of the excluded part + part.Evaluate(*this, window_begin, peer_begin, result, count, row_idx, WindowSegmentTreePart::LEFT); + + // 2. set up a second state for the right of the excluded part + if (!ltstate.right_part) { + ltstate.right_part = part.Copy(); + } + auto &right_part = *ltstate.right_part; + + // 3. evaluate the tree right of the excluded part + right_part.Evaluate(*this, peer_end, window_end, result, count, row_idx, WindowSegmentTreePart::RIGHT); + + // 4. combine the buffer state into the Segment Tree State + part.Combine(right_part, count); + } else { + part.Evaluate(*this, window_begin, window_end, result, count, row_idx, WindowSegmentTreePart::FULL); + } + + part.Finalize(result, count); + + // Set the validity mask on the invalid rows + + auto &rmask = FlatVector::Validity(result); + for (idx_t rid = 0, cur_row = row_idx; rid < count; ++rid, ++cur_row) { + auto begin = window_begin[rid]; + auto end = window_end[rid]; + if (begin >= end) { + rmask.SetInvalid(rid); + } + } +} + +void WindowSegmentTreePart::Evaluate(const WindowSegmentTree &tree, const idx_t *begins, const idx_t *ends, + Vector &result, idx_t count, idx_t row_idx, FramePart frame_part) { + + const auto cant_combine = (!aggr.function.combine || !tree.UseCombineAPI()); + auto fdata = FlatVector::GetData(statef); + + const auto exclude_mode = tree.exclude_mode; + const bool begin_on_curr_row = frame_part == FramePart::RIGHT && exclude_mode == WindowExcludeMode::CURRENT_ROW; + const bool end_on_curr_row = frame_part == FramePart::LEFT && exclude_mode == WindowExcludeMode::CURRENT_ROW; + // with EXCLUDE TIES, in addition to the frame part right of the peer group's end, we also need to consider the + // current row + const bool add_curr_row = frame_part == FramePart::RIGHT && exclude_mode == WindowExcludeMode::TIES; // First pass: aggregate the segment tree nodes // Share adjacent identical states // We do this first because we want to share only tree aggregations idx_t prev_begin = 1; idx_t prev_end = 0; - auto ldata = FlatVector::GetData(ltstate.statel); - auto pdata = FlatVector::GetData(ltstate.statep); + auto ldata = FlatVector::GetData(statel); + auto pdata = FlatVector::GetData(statep); data_ptr_t prev_state = nullptr; - for (idx_t rid = 0; rid < count; ++rid) { + for (idx_t rid = 0, cur_row = row_idx; rid < count; ++rid, ++cur_row) { auto state_ptr = fdata[rid]; aggr.function.initialize(state_ptr); @@ -556,23 +714,23 @@ void WindowSegmentTree::Evaluate(WindowAggregatorState &lstate, const idx_t *beg continue; } - auto begin = begins[rid]; - auto end = ends[rid]; + auto begin = begin_on_curr_row ? cur_row + 1 : begins[rid]; + auto end = end_on_curr_row ? cur_row : ends[rid]; if (begin >= end) { continue; } // Skip level 0 idx_t l_idx = 0; - for (; l_idx < levels_flat_start.size() + 1; l_idx++) { - idx_t parent_begin = begin / TREE_FANOUT; - idx_t parent_end = end / TREE_FANOUT; + for (; l_idx < tree.levels_flat_start.size() + 1; l_idx++) { + idx_t parent_begin = begin / tree.TREE_FANOUT; + idx_t parent_end = end / tree.TREE_FANOUT; if (prev_state && l_idx == 1 && begin == prev_begin && end == prev_end) { // Just combine the previous top level result - ldata[ltstate.flush_count] = prev_state; - pdata[ltstate.flush_count] = state_ptr; - if (++ltstate.flush_count >= STANDARD_VECTOR_SIZE) { - ltstate.FlushStates(true); + ldata[flush_count] = prev_state; + pdata[flush_count] = state_ptr; + if (++flush_count >= STANDARD_VECTOR_SIZE) { + FlushStates(true); } break; } @@ -585,72 +743,62 @@ void WindowSegmentTree::Evaluate(WindowAggregatorState &lstate, const idx_t *beg if (parent_begin == parent_end) { if (l_idx) { - ltstate.WindowSegmentValue(*this, l_idx, begin, end, state_ptr); + WindowSegmentValue(tree, l_idx, begin, end, state_ptr); } break; } - idx_t group_begin = parent_begin * TREE_FANOUT; + idx_t group_begin = parent_begin * tree.TREE_FANOUT; if (begin != group_begin) { if (l_idx) { - ltstate.WindowSegmentValue(*this, l_idx, begin, group_begin + TREE_FANOUT, state_ptr); + WindowSegmentValue(tree, l_idx, begin, group_begin + tree.TREE_FANOUT, state_ptr); } parent_begin++; } - idx_t group_end = parent_end * TREE_FANOUT; + idx_t group_end = parent_end * tree.TREE_FANOUT; if (end != group_end) { if (l_idx) { - ltstate.WindowSegmentValue(*this, l_idx, group_end, end, state_ptr); + WindowSegmentValue(tree, l_idx, group_end, end, state_ptr); } } begin = parent_begin; end = parent_end; } } - ltstate.FlushStates(true); + FlushStates(true); // Second pass: aggregate the ragged leaves // (or everything if we can't combine) - for (idx_t rid = 0; rid < count; ++rid) { + for (idx_t rid = 0, cur_row = row_idx; rid < count; ++rid, ++cur_row) { auto state_ptr = fdata[rid]; - const auto begin = begins[rid]; - const auto end = ends[rid]; + const auto begin = begin_on_curr_row ? cur_row + 1 : begins[rid]; + const auto end = end_on_curr_row ? cur_row : ends[rid]; + if (add_curr_row) { + WindowSegmentValue(tree, 0, cur_row, cur_row + 1, state_ptr); + } if (begin >= end) { continue; } // Aggregate everything at once if we can't combine states - idx_t parent_begin = begin / TREE_FANOUT; - idx_t parent_end = end / TREE_FANOUT; + idx_t parent_begin = begin / tree.TREE_FANOUT; + idx_t parent_end = end / tree.TREE_FANOUT; if (parent_begin == parent_end || cant_combine) { - ltstate.WindowSegmentValue(*this, 0, begin, end, state_ptr); + WindowSegmentValue(tree, 0, begin, end, state_ptr); continue; } - idx_t group_begin = parent_begin * TREE_FANOUT; + idx_t group_begin = parent_begin * tree.TREE_FANOUT; if (begin != group_begin) { - ltstate.WindowSegmentValue(*this, 0, begin, group_begin + TREE_FANOUT, state_ptr); + WindowSegmentValue(tree, 0, begin, group_begin + tree.TREE_FANOUT, state_ptr); parent_begin++; } - idx_t group_end = parent_end * TREE_FANOUT; + idx_t group_end = parent_end * tree.TREE_FANOUT; if (end != group_end) { - ltstate.WindowSegmentValue(*this, 0, group_end, end, state_ptr); - } - } - ltstate.FlushStates(false); - - ltstate.Finalize(result, count); - - // Set the validity mask on the invalid rows - auto &rmask = FlatVector::Validity(result); - for (idx_t rid = 0; rid < count; ++rid) { - const auto begin = begins[rid]; - const auto end = ends[rid]; - - if (begin >= end) { - rmask.SetInvalid(rid); + WindowSegmentValue(tree, 0, group_end, end, state_ptr); } } + FlushStates(false); } } // namespace duckdb diff --git a/src/duckdb/src/function/aggregate/distributive/count.cpp b/src/duckdb/src/function/aggregate/distributive/count.cpp index ec9d705bb..9631665c9 100644 --- a/src/duckdb/src/function/aggregate/distributive/count.cpp +++ b/src/duckdb/src/function/aggregate/distributive/count.cpp @@ -35,22 +35,26 @@ struct CountStarFunction : public BaseCountFunction { template static void Window(Vector inputs[], const ValidityMask &filter_mask, AggregateInputData &aggr_input_data, - idx_t input_count, data_ptr_t state, const FrameBounds &frame, const FrameBounds &prev, - Vector &result, idx_t rid, idx_t bias) { + idx_t input_count, data_ptr_t state, const vector &frames, Vector &result, + idx_t rid) { D_ASSERT(input_count == 0); + auto data = FlatVector::GetData(result); - const auto begin = frame.start; - const auto end = frame.end; - // Slice to any filtered rows - if (!filter_mask.AllValid()) { - RESULT_TYPE filtered = 0; + RESULT_TYPE total = 0; + for (const auto &frame : frames) { + const auto begin = frame.start; + const auto end = frame.end; + + // Slice to any filtered rows + if (filter_mask.AllValid()) { + total += end - begin; + continue; + } for (auto i = begin; i < end; ++i) { - filtered += filter_mask.RowIsValid(i); + total += filter_mask.RowIsValid(i); } - data[rid] = filtered; - } else { - data[rid] = end - begin; } + data[rid] = total; } }; diff --git a/src/duckdb/src/function/aggregate/sorted_aggregate_function.cpp b/src/duckdb/src/function/aggregate/sorted_aggregate_function.cpp index 6e5f21962..55e20470f 100644 --- a/src/duckdb/src/function/aggregate/sorted_aggregate_function.cpp +++ b/src/duckdb/src/function/aggregate/sorted_aggregate_function.cpp @@ -352,8 +352,8 @@ struct SortedAggregateFunction { } static void Window(Vector inputs[], const ValidityMask &filter_mask, AggregateInputData &aggr_input_data, - idx_t input_count, data_ptr_t state, const FrameBounds &frame, const FrameBounds &prev, - Vector &result, idx_t rid, idx_t bias) { + idx_t input_count, data_ptr_t state, const vector &frames, Vector &result, + idx_t rid) { throw InternalException("Sorted aggregates should not be generated for window clauses"); } diff --git a/src/duckdb/src/function/cast/array_casts.cpp b/src/duckdb/src/function/cast/array_casts.cpp new file mode 100644 index 000000000..e7a066dac --- /dev/null +++ b/src/duckdb/src/function/cast/array_casts.cpp @@ -0,0 +1,226 @@ +#include "duckdb/function/cast/cast_function_set.hpp" +#include "duckdb/function/cast/default_casts.hpp" +#include "duckdb/function/cast/bound_cast_data.hpp" +#include "duckdb/common/operator/cast_operators.hpp" + +namespace duckdb { + +unique_ptr ArrayBoundCastData::BindArrayToArrayCast(BindCastInput &input, const LogicalType &source, + const LogicalType &target) { + vector child_cast_info; + auto &source_child_type = ArrayType::GetChildType(source); + auto &result_child_type = ArrayType::GetChildType(target); + auto child_cast = input.GetCastFunction(source_child_type, result_child_type); + return make_uniq(std::move(child_cast)); +} + +static unique_ptr BindArrayToListCast(BindCastInput &input, const LogicalType &source, + const LogicalType &target) { + D_ASSERT(source.id() == LogicalTypeId::ARRAY); + D_ASSERT(target.id() == LogicalTypeId::LIST); + + vector child_cast_info; + auto &source_child_type = ArrayType::GetChildType(source); + auto &result_child_type = ListType::GetChildType(target); + auto child_cast = input.GetCastFunction(source_child_type, result_child_type); + return make_uniq(std::move(child_cast)); +} + +unique_ptr ArrayBoundCastData::InitArrayLocalState(CastLocalStateParameters ¶meters) { + auto &cast_data = parameters.cast_data->Cast(); + if (!cast_data.child_cast_info.init_local_state) { + return nullptr; + } + CastLocalStateParameters child_parameters(parameters, cast_data.child_cast_info.cast_data); + return cast_data.child_cast_info.init_local_state(child_parameters); +} + +//------------------------------------------------------------------------------ +// ARRAY -> ARRAY +//------------------------------------------------------------------------------ +static bool ArrayToArrayCast(Vector &source, Vector &result, idx_t count, CastParameters ¶meters) { + + auto source_array_size = ArrayType::GetSize(source.GetType()); + auto target_array_size = ArrayType::GetSize(result.GetType()); + if (source_array_size != target_array_size) { + // Cant cast between arrays of different sizes + auto msg = StringUtil::Format("Cannot cast array of size %u to array of size %u", source_array_size, + target_array_size); + HandleCastError::AssignError(msg, parameters.error_message); + if (!parameters.strict) { + // if this was a TRY_CAST, we know every row will fail, so just return null + result.SetVectorType(VectorType::CONSTANT_VECTOR); + ConstantVector::SetNull(result, true); + return false; + } + } + + auto &cast_data = parameters.cast_data->Cast(); + if (source.GetVectorType() == VectorType::CONSTANT_VECTOR) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + + if (ConstantVector::IsNull(source)) { + ConstantVector::SetNull(result, true); + } + + auto &source_cc = ArrayVector::GetEntry(source); + auto &result_cc = ArrayVector::GetEntry(result); + + // If the array vector is constant, the child vector must be flat (or constant if array size is 1) + D_ASSERT(source_cc.GetVectorType() == VectorType::FLAT_VECTOR || source_array_size == 1); + + CastParameters child_parameters(parameters, cast_data.child_cast_info.cast_data, parameters.local_state); + bool all_ok = cast_data.child_cast_info.function(source_cc, result_cc, source_array_size, child_parameters); + return all_ok; + } else { + // Flatten if not constant + source.Flatten(count); + result.SetVectorType(VectorType::FLAT_VECTOR); + + FlatVector::SetValidity(result, FlatVector::Validity(source)); + auto &source_cc = ArrayVector::GetEntry(source); + auto &result_cc = ArrayVector::GetEntry(result); + + CastParameters child_parameters(parameters, cast_data.child_cast_info.cast_data, parameters.local_state); + bool all_ok = + cast_data.child_cast_info.function(source_cc, result_cc, count * source_array_size, child_parameters); + return all_ok; + } +} + +//------------------------------------------------------------------------------ +// ARRAY -> VARCHAR +//------------------------------------------------------------------------------ +static bool ArrayToVarcharCast(Vector &source, Vector &result, idx_t count, CastParameters ¶meters) { + auto is_constant = source.GetVectorType() == VectorType::CONSTANT_VECTOR; + + auto size = ArrayType::GetSize(source.GetType()); + Vector varchar_list(LogicalType::ARRAY(LogicalType::VARCHAR, size), count); + ArrayToArrayCast(source, varchar_list, count, parameters); + + varchar_list.Flatten(count); + auto &validity = FlatVector::Validity(varchar_list); + auto &child = ArrayVector::GetEntry(varchar_list); + + child.Flatten(count); + auto &child_validity = FlatVector::Validity(child); + + auto in_data = FlatVector::GetData(child); + auto out_data = FlatVector::GetData(result); + + static constexpr const idx_t SEP_LENGTH = 2; + static constexpr const idx_t NULL_LENGTH = 4; + + for (idx_t i = 0; i < count; i++) { + if (!validity.RowIsValid(i)) { + FlatVector::SetNull(result, i, true); + continue; + } + + // First pass, compute the length + idx_t array_varchar_length = 2; + for (idx_t j = 0; j < size; j++) { + auto elem_idx = (i * size) + j; + auto elem = in_data[elem_idx]; + if (j > 0) { + array_varchar_length += SEP_LENGTH; + } + array_varchar_length += child_validity.RowIsValid(elem_idx) ? elem.GetSize() : NULL_LENGTH; + } + + out_data[i] = StringVector::EmptyString(result, array_varchar_length); + auto dataptr = out_data[i].GetDataWriteable(); + auto offset = 0; + dataptr[offset++] = '['; + + // Second pass, write the actual data + for (idx_t j = 0; j < size; j++) { + auto elem_idx = (i * size) + j; + auto elem = in_data[elem_idx]; + if (j > 0) { + memcpy(dataptr + offset, ", ", SEP_LENGTH); + offset += SEP_LENGTH; + } + if (child_validity.RowIsValid(elem_idx)) { + auto len = elem.GetSize(); + memcpy(dataptr + offset, elem.GetData(), len); + offset += len; + } else { + memcpy(dataptr + offset, "NULL", NULL_LENGTH); + offset += NULL_LENGTH; + } + } + dataptr[offset++] = ']'; + out_data[i].Finalize(); + } + + if (is_constant) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } + + return true; +} + +//------------------------------------------------------------------------------ +// ARRAY -> LIST +//------------------------------------------------------------------------------ +static bool ArrayToListCast(Vector &source, Vector &result, idx_t count, CastParameters ¶meters) { + auto &cast_data = parameters.cast_data->Cast(); + + // FIXME: dont flatten + source.Flatten(count); + if (count == 1) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } + auto array_size = ArrayType::GetSize(source.GetType()); + auto child_count = count * array_size; + + ListVector::Reserve(result, child_count); + ListVector::SetListSize(result, child_count); + + auto &source_child = ArrayVector::GetEntry(source); + auto &result_child = ListVector::GetEntry(result); + + CastParameters child_parameters(parameters, cast_data.child_cast_info.cast_data, parameters.local_state); + bool all_ok = cast_data.child_cast_info.function(source_child, result_child, child_count, child_parameters); + + auto list_data = ListVector::GetData(result); + for (idx_t i = 0; i < count; i++) { + if (FlatVector::IsNull(source, i)) { + FlatVector::SetNull(result, i, true); + continue; + } + + list_data[i].offset = i * array_size; + list_data[i].length = array_size; + } + + if (count == 1) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } + + return all_ok; +} + +BoundCastInfo DefaultCasts::ArrayCastSwitch(BindCastInput &input, const LogicalType &source, + const LogicalType &target) { + switch (target.id()) { + case LogicalTypeId::VARCHAR: { + auto size = ArrayType::GetSize(source); + return BoundCastInfo( + ArrayToVarcharCast, + ArrayBoundCastData::BindArrayToArrayCast(input, source, LogicalType::ARRAY(LogicalType::VARCHAR, size)), + ArrayBoundCastData::InitArrayLocalState); + } + case LogicalTypeId::ARRAY: + return BoundCastInfo(ArrayToArrayCast, ArrayBoundCastData::BindArrayToArrayCast(input, source, target), + ArrayBoundCastData::InitArrayLocalState); + case LogicalTypeId::LIST: + return BoundCastInfo(ArrayToListCast, BindArrayToListCast(input, source, target), + ArrayBoundCastData::InitArrayLocalState); + default: + return DefaultCasts::TryVectorNullCast; + }; +} + +} // namespace duckdb diff --git a/src/duckdb/src/function/cast/cast_function_set.cpp b/src/duckdb/src/function/cast/cast_function_set.cpp index cc152e623..6e239f8fe 100644 --- a/src/duckdb/src/function/cast/cast_function_set.cpp +++ b/src/duckdb/src/function/cast/cast_function_set.cpp @@ -91,6 +91,8 @@ static auto RelaxedTypeMatch(type_map_t &map, const LogicalType return map.end(); case LogicalTypeId::UNION: return map.find(LogicalType::UNION({{"any", LogicalType::ANY}})); + case LogicalTypeId::ARRAY: + return map.find(LogicalType::ARRAY(LogicalType::ANY)); default: return map.find(LogicalType::ANY); } diff --git a/src/duckdb/src/function/cast/default_casts.cpp b/src/duckdb/src/function/cast/default_casts.cpp index 1b6bff010..60c5eb526 100644 --- a/src/duckdb/src/function/cast/default_casts.cpp +++ b/src/duckdb/src/function/cast/default_casts.cpp @@ -137,6 +137,8 @@ BoundCastInfo DefaultCasts::GetDefaultCastFunction(BindCastInput &input, const L return UnionCastSwitch(input, source, target); case LogicalTypeId::ENUM: return EnumCastSwitch(input, source, target); + case LogicalTypeId::ARRAY: + return ArrayCastSwitch(input, source, target); case LogicalTypeId::AGGREGATE_STATE: return AggregateStateToBlobCast; default: diff --git a/src/duckdb/src/function/cast/list_casts.cpp b/src/duckdb/src/function/cast/list_casts.cpp index a1c20b4c0..69cd530c0 100644 --- a/src/duckdb/src/function/cast/list_casts.cpp +++ b/src/duckdb/src/function/cast/list_casts.cpp @@ -1,6 +1,7 @@ #include "duckdb/function/cast/default_casts.hpp" #include "duckdb/function/cast/cast_function_set.hpp" #include "duckdb/function/cast/bound_cast_data.hpp" +#include "duckdb/common/operator/cast_operators.hpp" namespace duckdb { @@ -13,6 +14,15 @@ unique_ptr ListBoundCastData::BindListToListCast(BindCastInput &i return make_uniq(std::move(child_cast)); } +static unique_ptr BindListToArrayCast(BindCastInput &input, const LogicalType &source, + const LogicalType &target) { + vector child_cast_info; + auto &source_child_type = ListType::GetChildType(source); + auto &result_child_type = ArrayType::GetChildType(target); + auto child_cast = input.GetCastFunction(source_child_type, result_child_type); + return make_uniq(std::move(child_cast)); +} + unique_ptr ListBoundCastData::InitListLocalState(CastLocalStateParameters ¶meters) { auto &cast_data = parameters.cast_data->Cast(); if (!cast_data.child_cast_info.init_local_state) { @@ -121,6 +131,113 @@ static bool ListToVarcharCast(Vector &source, Vector &result, idx_t count, CastP return true; } +static bool ListToArrayCast(Vector &source, Vector &result, idx_t count, CastParameters ¶meters) { + auto &cast_data = parameters.cast_data->Cast(); + auto array_size = ArrayType::GetSize(result.GetType()); + + // only handle constant and flat vectors here for now + if (source.GetVectorType() == VectorType::CONSTANT_VECTOR) { + result.SetVectorType(source.GetVectorType()); + if (ConstantVector::IsNull(source)) { + ConstantVector::SetNull(result, true); + return true; + } + + auto ldata = ConstantVector::GetData(source)[0]; + if (!ConstantVector::IsNull(source) && ldata.length != array_size) { + // Cant cast to array, list size mismatch + auto msg = StringUtil::Format("Cannot cast list with length %llu to array with length %u", ldata.length, + array_size); + HandleCastError::AssignError(msg, parameters.error_message); + ConstantVector::SetNull(result, true); + return false; + } + + auto &source_cc = ListVector::GetEntry(source); + auto &result_cc = ArrayVector::GetEntry(result); + + // Since the list was constant, there can only be one sequence of data in the child vector + CastParameters child_parameters(parameters, cast_data.child_cast_info.cast_data, parameters.local_state); + bool all_succeeded = cast_data.child_cast_info.function(source_cc, result_cc, array_size, child_parameters); + return all_succeeded; + } else { + source.Flatten(count); + result.SetVectorType(VectorType::FLAT_VECTOR); + + auto child_type = ArrayType::GetChildType(result.GetType()); + auto &source_cc = ListVector::GetEntry(source); + auto &result_cc = ArrayVector::GetEntry(result); + auto ldata = FlatVector::GetData(source); + + bool all_lengths_match = true; + + // Now, here's where things get funky. + // We need to slice out the data from the child vector, since the list + // wont store child data for null entries, which is the case for arrays. + // So we need to preserve the "gaps" for null entries when we copy into + // the result vector. In short 'len(source_child) <= len(result_child)' + // depending on how many NULL's the source has. so we "unslice" the + // child data so that len(payload_vector) == len(result_child). + + auto child_count = array_size * count; + Vector payload_vector(source_cc.GetType(), child_count); + SelectionVector sel(child_count); + + for (idx_t i = 0; i < count; i++) { + // If the list is null, set the entire array to null + if (FlatVector::IsNull(source, i)) { + FlatVector::SetNull(result, i, true); + for (idx_t array_elem = 0; array_elem < array_size; array_elem++) { + FlatVector::SetNull(payload_vector, i * array_size + array_elem, true); + // just select the first value, it won't be used anyway + sel.set_index(i * array_size + array_elem, 0); + } + } else if (ldata[i].length != array_size) { + if (all_lengths_match) { + // Cant cast to array, list size mismatch + all_lengths_match = false; + auto msg = StringUtil::Format("Cannot cast list with length %llu to array with length %u", + ldata[i].length, array_size); + HandleCastError::AssignError(msg, parameters.error_message); + } + FlatVector::SetNull(result, i, true); + for (idx_t array_elem = 0; array_elem < array_size; array_elem++) { + FlatVector::SetNull(payload_vector, i * array_size + array_elem, true); + // just select the first value, it won't be used anyway + sel.set_index(i * array_size + array_elem, 0); + } + } else { + // Set the selection vector to point to the correct offsets + for (idx_t array_elem = 0; array_elem < array_size; array_elem++) { + sel.set_index(i * array_size + array_elem, ldata[i].offset + array_elem); + } + } + } + + // Perform the actual copy + VectorOperations::Copy(source_cc, payload_vector, sel, child_count, 0, 0); + + // Take a last pass and null out the child elems + for (idx_t i = 0; i < count; i++) { + if (FlatVector::IsNull(source, i) || FlatVector::IsNull(result, i)) { + for (idx_t array_elem = 0; array_elem < array_size; array_elem++) { + FlatVector::SetNull(payload_vector, i * array_size + array_elem, true); + } + } + } + + CastParameters child_parameters(parameters, cast_data.child_cast_info.cast_data, parameters.local_state); + bool child_succeeded = + cast_data.child_cast_info.function(payload_vector, result_cc, child_count, child_parameters); + + if (!child_succeeded) { + HandleCastError::AssignError(*child_parameters.error_message, parameters.error_message); + } + + return all_lengths_match && child_succeeded; + } +} + BoundCastInfo DefaultCasts::ListCastSwitch(BindCastInput &input, const LogicalType &source, const LogicalType &target) { switch (target.id()) { case LogicalTypeId::LIST: @@ -131,6 +248,9 @@ BoundCastInfo DefaultCasts::ListCastSwitch(BindCastInput &input, const LogicalTy ListToVarcharCast, ListBoundCastData::BindListToListCast(input, source, LogicalType::LIST(LogicalType::VARCHAR)), ListBoundCastData::InitListLocalState); + case LogicalTypeId::ARRAY: + return BoundCastInfo(ListToArrayCast, BindListToArrayCast(input, source, target), + ListBoundCastData::InitListLocalState); default: return DefaultCasts::TryVectorNullCast; } diff --git a/src/duckdb/src/function/cast/string_cast.cpp b/src/duckdb/src/function/cast/string_cast.cpp index 5ef31d904..dc4f0c76f 100644 --- a/src/duckdb/src/function/cast/string_cast.cpp +++ b/src/duckdb/src/function/cast/string_cast.cpp @@ -333,6 +333,81 @@ bool VectorStringToMap::StringToNestedTypeCastLoop(const string_t *source_data, return all_converted; } +//===--------------------------------------------------------------------===// +// string -> array casting +//===--------------------------------------------------------------------===// +bool VectorStringToArray::StringToNestedTypeCastLoop(const string_t *source_data, ValidityMask &source_mask, + Vector &result, ValidityMask &result_mask, idx_t count, + CastParameters ¶meters, const SelectionVector *sel) { + idx_t array_size = ArrayType::GetSize(result.GetType()); + bool all_lengths_match = true; + + for (idx_t i = 0; i < count; i++) { + idx_t idx = i; + if (sel) { + idx = sel->get_index(i); + } + if (!source_mask.RowIsValid(idx)) { + continue; + } + auto str_array_size = VectorStringToList::CountPartsList(source_data[idx]); + if (array_size != str_array_size) { + if (all_lengths_match) { + all_lengths_match = false; + auto msg = + StringUtil::Format("Type VARCHAR with value '%s' can't be cast to the destination type ARRAY[%u]" + ", the size of the array must match the destination type", + source_data[idx].GetString(), array_size); + if (parameters.strict) { + throw CastException(msg); + } + HandleCastError::AssignError(msg, parameters.error_message); + } + result_mask.SetInvalid(i); + } + } + + auto child_count = array_size * count; + Vector varchar_vector(LogicalType::VARCHAR, child_count); + auto child_data = FlatVector::GetData(varchar_vector); + + bool all_converted = true; + idx_t total = 0; + for (idx_t i = 0; i < count; i++) { + idx_t idx = i; + if (sel) { + idx = sel->get_index(i); + } + + if (!source_mask.RowIsValid(idx) || !result_mask.RowIsValid(i)) { + // The source is null, or there was a size-mismatch above, so dont try to split the string + result_mask.SetInvalid(i); + + // Null the entire array + for (idx_t j = 0; j < array_size; j++) { + FlatVector::SetNull(varchar_vector, idx * array_size + j, true); + } + + total += array_size; + continue; + } + + if (!VectorStringToList::SplitStringList(source_data[idx], child_data, total, varchar_vector)) { + auto text = StringUtil::Format("Type VARCHAR with value '%s' can't be cast to the destination type ARRAY", + source_data[idx].GetString()); + HandleVectorCastError::Operation(text, result_mask, idx, parameters.error_message, all_converted); + } + } + D_ASSERT(total == child_count); + + auto &result_child = ArrayVector::GetEntry(result); + auto &cast_data = parameters.cast_data->Cast(); + CastParameters child_parameters(parameters, cast_data.child_cast_info.cast_data, parameters.local_state); + bool cast_result = cast_data.child_cast_info.function(varchar_vector, result_child, child_count, child_parameters); + + return all_lengths_match && cast_result && all_converted; +} + template bool StringToNestedTypeCast(Vector &source, Vector &result, idx_t count, CastParameters ¶meters) { D_ASSERT(source.GetType().id() == LogicalTypeId::VARCHAR); @@ -398,6 +473,12 @@ BoundCastInfo DefaultCasts::StringCastSwitch(BindCastInput &input, const Logical &StringToNestedTypeCast, ListBoundCastData::BindListToListCast(input, LogicalType::LIST(LogicalType::VARCHAR), target), ListBoundCastData::InitListLocalState); + case LogicalTypeId::ARRAY: + // the second argument allows for a secondary casting function to be passed in the CastParameters + return BoundCastInfo( + &StringToNestedTypeCast, + ArrayBoundCastData::BindArrayToArrayCast(input, LogicalType::ARRAY(LogicalType::VARCHAR), target), + ArrayBoundCastData::InitArrayLocalState); case LogicalTypeId::STRUCT: return BoundCastInfo(&StringToNestedTypeCast, StructBoundCastData::BindStructToStructCast(input, InitVarcharStructType(target), target), diff --git a/src/duckdb/src/function/cast/time_casts.cpp b/src/duckdb/src/function/cast/time_casts.cpp index 3586247a4..acd12798f 100644 --- a/src/duckdb/src/function/cast/time_casts.cpp +++ b/src/duckdb/src/function/cast/time_casts.cpp @@ -114,6 +114,9 @@ BoundCastInfo DefaultCasts::TimestampNsCastSwitch(BindCastInput &input, const Lo case LogicalTypeId::VARCHAR: // timestamp (ns) to varchar return BoundCastInfo(&VectorCastHelpers::StringCast); + case LogicalTypeId::DATE: + // timestamp (ns) to date + return BoundCastInfo(&VectorCastHelpers::TemplatedCastLoop); case LogicalTypeId::TIMESTAMP: // timestamp (ns) to timestamp (us) return BoundCastInfo( @@ -130,6 +133,9 @@ BoundCastInfo DefaultCasts::TimestampMsCastSwitch(BindCastInput &input, const Lo case LogicalTypeId::VARCHAR: // timestamp (ms) to varchar return BoundCastInfo(&VectorCastHelpers::StringCast); + case LogicalTypeId::DATE: + // timestamp (ms) to date + return BoundCastInfo(&VectorCastHelpers::TemplatedCastLoop); case LogicalTypeId::TIMESTAMP: // timestamp (ms) to timestamp (us) return BoundCastInfo( @@ -148,8 +154,12 @@ BoundCastInfo DefaultCasts::TimestampSecCastSwitch(BindCastInput &input, const L // now switch on the result type switch (target.id()) { case LogicalTypeId::VARCHAR: - // timestamp (sec) to varchar + // timestamp (s) to varchar return BoundCastInfo(&VectorCastHelpers::StringCast); + case LogicalTypeId::DATE: + // timestamp (s) to date + return BoundCastInfo( + &VectorCastHelpers::TemplatedCastLoop); case LogicalTypeId::TIMESTAMP_MS: // timestamp (s) to timestamp (ms) return BoundCastInfo( diff --git a/src/duckdb/src/function/cast_rules.cpp b/src/duckdb/src/function/cast_rules.cpp index f4ffb84c9..239a9e22b 100644 --- a/src/duckdb/src/function/cast_rules.cpp +++ b/src/duckdb/src/function/cast_rules.cpp @@ -23,6 +23,7 @@ static int64_t TargetTypeCost(const LogicalType &type) { case LogicalTypeId::MAP: case LogicalTypeId::LIST: case LogicalTypeId::UNION: + case LogicalTypeId::ARRAY: return 160; default: return 110; @@ -216,6 +217,33 @@ int64_t CastRules::ImplicitCast(const LogicalType &from, const LogicalType &to) } return child_cost; } + if (from.id() == LogicalTypeId::ARRAY && to.id() == LogicalTypeId::ARRAY) { + // Arrays can be cast if their child types can be cast and the source and target has the same size + // or the target type has a unknown (any) size. + auto from_size = ArrayType::GetSize(from); + auto to_size = ArrayType::GetSize(to); + auto to_is_any_size = ArrayType::IsAnySize(to); + if (from_size == to_size || to_is_any_size) { + auto child_cost = ImplicitCast(ArrayType::GetChildType(from), ArrayType::GetChildType(to)); + if (child_cost >= 100) { + // subtract one from the cost because we prefer ARRAY[X] -> ARRAY[VARCHAR] over ARRAY[X] -> VARCHAR + child_cost--; + } + return child_cost; + } + return -1; // Not possible if the sizes are different + } + if (from.id() == LogicalTypeId::ARRAY && to.id() == LogicalTypeId::LIST) { + // Arrays can be cast to lists for the cost of casting the child type + // add 1 because we prefer ARRAY->ARRAY casts over ARRAY->LIST casts + return ImplicitCast(ArrayType::GetChildType(from), ListType::GetChildType(to)) + 1; + } + if (from.id() == LogicalTypeId::LIST && (to.id() == LogicalTypeId::ARRAY && !ArrayType::IsAnySize(to))) { + // Lists can be cast to arrays for the cost of casting the child type, if the target size is known + // there is no way for us to resolve the size at bind-time without inspecting the list values. + // TODO: if we can access the expression we could resolve the size if the list is constant. + return ImplicitCast(ListType::GetChildType(from), ArrayType::GetChildType(to)); + } if (from.id() == to.id()) { // arguments match: do nothing return 0; diff --git a/src/duckdb/src/function/function_binder.cpp b/src/duckdb/src/function/function_binder.cpp index 68d7ef812..b7b3fa54e 100644 --- a/src/duckdb/src/function/function_binder.cpp +++ b/src/duckdb/src/function/function_binder.cpp @@ -221,6 +221,9 @@ LogicalTypeComparisonResult RequiresCast(const LogicalType &source_type, const L if (source_type.id() == LogicalTypeId::LIST && target_type.id() == LogicalTypeId::LIST) { return RequiresCast(ListType::GetChildType(source_type), ListType::GetChildType(target_type)); } + if (source_type.id() == LogicalTypeId::ARRAY && target_type.id() == LogicalTypeId::ARRAY) { + return RequiresCast(ArrayType::GetChildType(source_type), ArrayType::GetChildType(target_type)); + } return LogicalTypeComparisonResult::DIFFERENT_TYPES; } diff --git a/src/duckdb/src/function/function_set.cpp b/src/duckdb/src/function/function_set.cpp index dbbfdfa8b..41d54c886 100644 --- a/src/duckdb/src/function/function_set.cpp +++ b/src/duckdb/src/function/function_set.cpp @@ -49,7 +49,7 @@ AggregateFunction AggregateFunctionSet::GetFunctionByArguments(ClientContext &co } bool is_prefix = true; for (idx_t k = 0; k < arguments.size(); k++) { - if (arguments[k].id() != func.arguments[k].id()) { + if (arguments[k] != func.arguments[k]) { is_prefix = false; break; } diff --git a/src/duckdb/src/function/pragma/pragma_queries.cpp b/src/duckdb/src/function/pragma/pragma_queries.cpp index 781efebb1..4e45d0dd9 100644 --- a/src/duckdb/src/function/pragma/pragma_queries.cpp +++ b/src/duckdb/src/function/pragma/pragma_queries.cpp @@ -194,10 +194,6 @@ string PragmaMetadataInfo(ClientContext &context, const FunctionParameters ¶ return "SELECT * FROM pragma_metadata_info();"; } -string PragmaUserAgent(ClientContext &context, const FunctionParameters ¶meters) { - return "SELECT * FROM pragma_user_agent()"; -} - void PragmaQueries::RegisterFunction(BuiltinFunctions &set) { set.AddFunction(PragmaFunction::PragmaCall("table_info", PragmaTableInfo, {LogicalType::VARCHAR})); set.AddFunction(PragmaFunction::PragmaCall("storage_info", PragmaStorageInfo, {LogicalType::VARCHAR})); @@ -214,7 +210,6 @@ void PragmaQueries::RegisterFunction(BuiltinFunctions &set) { set.AddFunction(PragmaFunction::PragmaStatement("functions", PragmaFunctionsQuery)); set.AddFunction(PragmaFunction::PragmaCall("import_database", PragmaImportDatabase, {LogicalType::VARCHAR})); set.AddFunction(PragmaFunction::PragmaStatement("all_profiling_output", PragmaAllProfiling)); - set.AddFunction(PragmaFunction::PragmaStatement("user_agent", PragmaUserAgent)); } } // namespace duckdb diff --git a/src/duckdb/src/function/scalar/list/contains_or_position.cpp b/src/duckdb/src/function/scalar/list/contains_or_position.cpp index 064f4c28e..3b9614359 100644 --- a/src/duckdb/src/function/scalar/list/contains_or_position.cpp +++ b/src/duckdb/src/function/scalar/list/contains_or_position.cpp @@ -2,6 +2,7 @@ #include "duckdb/planner/expression/bound_function_expression.hpp" #include "duckdb/planner/expression_binder.hpp" #include "duckdb/common/operator/comparison_operators.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" namespace duckdb { @@ -20,6 +21,9 @@ static unique_ptr ListContainsOrPositionBind(ClientContext &contex vector> &arguments) { D_ASSERT(bound_function.arguments.size() == 2); + // If the first argument is an array, cast it to a list + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + const auto &list = arguments[0]->return_type; // change to list const auto &value = arguments[1]->return_type; if (list.id() == LogicalTypeId::UNKNOWN) { diff --git a/src/duckdb/src/function/scalar/list/list_concat.cpp b/src/duckdb/src/function/scalar/list/list_concat.cpp index 0898f513f..d992bd951 100644 --- a/src/duckdb/src/function/scalar/list/list_concat.cpp +++ b/src/duckdb/src/function/scalar/list/list_concat.cpp @@ -3,6 +3,7 @@ #include "duckdb/planner/expression/bound_function_expression.hpp" #include "duckdb/planner/expression/bound_parameter_expression.hpp" #include "duckdb/planner/expression_binder.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" namespace duckdb { @@ -76,8 +77,13 @@ static unique_ptr ListConcatBind(ClientContext &context, ScalarFun vector> &arguments) { D_ASSERT(bound_function.arguments.size() == 2); + // if either argument is an array, we cast it to a list + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + arguments[1] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[1])); + auto &lhs = arguments[0]->return_type; auto &rhs = arguments[1]->return_type; + if (lhs.id() == LogicalTypeId::UNKNOWN || rhs.id() == LogicalTypeId::UNKNOWN) { throw ParameterNotResolvedException(); } else if (lhs.id() == LogicalTypeId::SQLNULL || rhs.id() == LogicalTypeId::SQLNULL) { diff --git a/src/duckdb/src/function/scalar/list/list_extract.cpp b/src/duckdb/src/function/scalar/list/list_extract.cpp index e7d28d524..cb0ffb5ad 100644 --- a/src/duckdb/src/function/scalar/list/list_extract.cpp +++ b/src/duckdb/src/function/scalar/list/list_extract.cpp @@ -7,6 +7,7 @@ #include "duckdb/function/scalar/string_functions.hpp" #include "duckdb/parser/expression/bound_expression.hpp" #include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" #include "duckdb/storage/statistics/list_stats.hpp" namespace duckdb { @@ -203,9 +204,14 @@ static void ListExtractFunction(DataChunk &args, ExpressionState &state, Vector static unique_ptr ListExtractBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { D_ASSERT(bound_function.arguments.size() == 2); + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + D_ASSERT(LogicalTypeId::LIST == arguments[0]->return_type.id()); // list extract returns the child type of the list as return type - bound_function.return_type = ListType::GetChildType(arguments[0]->return_type); + auto child_type = ListType::GetChildType(arguments[0]->return_type); + + bound_function.return_type = child_type; + bound_function.arguments[0] = LogicalType::LIST(child_type); return make_uniq(bound_function.return_type); } diff --git a/src/duckdb/src/function/scalar/list/list_resize.cpp b/src/duckdb/src/function/scalar/list/list_resize.cpp index 107bb2df6..ab5c5acc9 100644 --- a/src/duckdb/src/function/scalar/list/list_resize.cpp +++ b/src/duckdb/src/function/scalar/list/list_resize.cpp @@ -2,6 +2,7 @@ #include "duckdb/function/scalar/nested_functions.hpp" #include "duckdb/function/scalar_function.hpp" #include "duckdb/function/built_in_functions.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" namespace duckdb { @@ -115,6 +116,9 @@ static unique_ptr ListResizeBind(ClientContext &context, ScalarFun D_ASSERT(bound_function.arguments.size() == 2 || arguments.size() == 3); bound_function.arguments[1] = LogicalType::UBIGINT; + // If the first argument is an array, cast it to a list + arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); + // first argument is constant NULL if (arguments[0]->return_type == LogicalType::SQLNULL) { bound_function.arguments[0] = LogicalType::SQLNULL; diff --git a/src/duckdb/src/function/scalar/list/list_select.cpp b/src/duckdb/src/function/scalar/list/list_select.cpp new file mode 100644 index 000000000..1da6e30ff --- /dev/null +++ b/src/duckdb/src/function/scalar/list/list_select.cpp @@ -0,0 +1,176 @@ +#include "duckdb/common/types/data_chunk.hpp" +#include "duckdb/function/scalar/nested_functions.hpp" +#include "duckdb/planner/expression_binder.hpp" +#include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/planner/expression/bound_parameter_expression.hpp" + +namespace duckdb { + +struct SetSelectionVectorSelect { + static void SetSelectionVector(SelectionVector &selection_vector, ValidityMask &validity_mask, + ValidityMask &input_validity, Vector &selection_entry, idx_t child_idx, + idx_t &target_offset, idx_t selection_offset, idx_t input_offset, + idx_t target_length) { + idx_t sel_idx = selection_entry.GetValue(selection_offset + child_idx).GetValue() - 1; + if (sel_idx < target_length) { + selection_vector.set_index(target_offset, input_offset + sel_idx); + if (!input_validity.RowIsValid(input_offset + sel_idx)) { + validity_mask.SetInvalid(target_offset); + } + } else { + selection_vector.set_index(target_offset, 0); + validity_mask.SetInvalid(target_offset); + } + target_offset++; + } + + static void GetResultLength(DataChunk &args, idx_t &result_length, const list_entry_t *selection_data, + Vector selection_entry, idx_t selection_idx) { + result_length += selection_data[selection_idx].length; + } +}; + +struct SetSelectionVectorWhere { + static void SetSelectionVector(SelectionVector &selection_vector, ValidityMask &validity_mask, + ValidityMask &input_validity, Vector &selection_entry, idx_t child_idx, + idx_t &target_offset, idx_t selection_offset, idx_t input_offset, + idx_t target_length) { + if (!selection_entry.GetValue(selection_offset + child_idx).GetValue()) { + return; + } + + selection_vector.set_index(target_offset, input_offset + child_idx); + if (!input_validity.RowIsValid(input_offset + child_idx)) { + validity_mask.SetInvalid(target_offset); + } + target_offset++; + } + + static void GetResultLength(DataChunk &args, idx_t &result_length, const list_entry_t *selection_data, + Vector selection_entry, idx_t selection_idx) { + for (idx_t child_idx = 0; child_idx < selection_data[selection_idx].length; child_idx++) { + if (selection_entry.GetValue(selection_data[selection_idx].offset + child_idx).IsNull()) { + throw InvalidInputException("NULLs are not allowed as list elements in the second input parameter."); + } + if (selection_entry.GetValue(selection_data[selection_idx].offset + child_idx).GetValue()) { + result_length++; + } + } + } +}; + +template +static void ListSelectFunction(DataChunk &args, ExpressionState &state, Vector &result) { + D_ASSERT(args.data.size() == 2); + Vector &list = args.data[0]; + Vector &selection_list = args.data[1]; + idx_t count = args.size(); + + list_entry_t *result_data; + result_data = FlatVector::GetData(result); + auto &result_entry = ListVector::GetEntry(result); + + UnifiedVectorFormat selection_lists; + selection_list.ToUnifiedFormat(count, selection_lists); + auto selection_lists_data = UnifiedVectorFormat::GetData(selection_lists); + auto &selection_entry = ListVector::GetEntry(selection_list); + + UnifiedVectorFormat input_list; + list.ToUnifiedFormat(count, input_list); + auto input_lists_data = UnifiedVectorFormat::GetData(input_list); + auto &input_entry = ListVector::GetEntry(list); + auto &input_validity = FlatVector::Validity(input_entry); + + idx_t result_length = 0; + for (idx_t i = 0; i < count; i++) { + idx_t input_idx = input_list.sel->get_index(i); + idx_t selection_idx = selection_lists.sel->get_index(i); + if (input_list.validity.RowIsValid(input_idx) && selection_lists.validity.RowIsValid(selection_idx)) { + OP::GetResultLength(args, result_length, selection_lists_data, selection_entry, selection_idx); + } + } + + ListVector::Reserve(result, result_length); + SelectionVector result_selection_vec = SelectionVector(result_length); + ValidityMask entry_validity_mask = ValidityMask(result_length); + ValidityMask &result_validity_mask = FlatVector::Validity(result); + + idx_t offset = 0; + for (idx_t j = 0; j < count; j++) { + // Get length and offset of selection list for current output row + auto selection_list_idx = selection_lists.sel->get_index(j); + idx_t selection_len = 0; + idx_t selection_offset = 0; + if (selection_lists.validity.RowIsValid(selection_list_idx)) { + selection_len = selection_lists_data[selection_list_idx].length; + selection_offset = selection_lists_data[selection_list_idx].offset; + } else { + result_validity_mask.SetInvalid(j); + continue; + } + // Get length and offset of input list for current output row + auto input_list_idx = input_list.sel->get_index(j); + idx_t input_length = 0; + idx_t input_offset = 0; + if (input_list.validity.RowIsValid(input_list_idx)) { + input_length = input_lists_data[input_list_idx].length; + input_offset = input_lists_data[input_list_idx].offset; + } else { + result_validity_mask.SetInvalid(j); + continue; + } + result_data[j].offset = offset; + // Set all selected values in the result + for (idx_t child_idx = 0; child_idx < selection_len; child_idx++) { + if (selection_entry.GetValue(selection_offset + child_idx).IsNull()) { + throw InvalidInputException("NULLs are not allowed as list elements in the second input parameter."); + } + OP::SetSelectionVector(result_selection_vec, entry_validity_mask, input_validity, selection_entry, + child_idx, offset, selection_offset, input_offset, input_length); + } + result_data[j].length = offset - result_data[j].offset; + } + result_entry.Slice(input_entry, result_selection_vec, count); + result_entry.Flatten(offset); + ListVector::SetListSize(result, offset); + FlatVector::SetValidity(result_entry, entry_validity_mask); + result.SetVectorType(args.AllConstant() ? VectorType::CONSTANT_VECTOR : VectorType::FLAT_VECTOR); +} + +static unique_ptr ListSelectBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + D_ASSERT(bound_function.arguments.size() == 2); + LogicalType child_type; + if (arguments[0]->return_type == LogicalTypeId::UNKNOWN || arguments[1]->return_type == LogicalTypeId::UNKNOWN) { + bound_function.arguments[0] = LogicalTypeId::UNKNOWN; + bound_function.return_type = LogicalType::SQLNULL; + return make_uniq(bound_function.return_type); + } + + D_ASSERT(LogicalTypeId::LIST == arguments[0]->return_type.id() || + LogicalTypeId::SQLNULL == arguments[0]->return_type.id()); + D_ASSERT(LogicalTypeId::LIST == arguments[1]->return_type.id() || + LogicalTypeId::SQLNULL == arguments[1]->return_type.id()); + + bound_function.return_type = arguments[0]->return_type; + return make_uniq(bound_function.return_type); +} +ScalarFunction ListWhereFun::GetFunction() { + auto fun = ScalarFunction({LogicalType::LIST(LogicalTypeId::ANY), LogicalType::LIST(LogicalType::BOOLEAN)}, + LogicalType::LIST(LogicalTypeId::ANY), ListSelectFunction, + ListSelectBind); + return fun; +} + +ScalarFunction ListSelectFun::GetFunction() { + auto fun = ScalarFunction({LogicalType::LIST(LogicalTypeId::ANY), LogicalType::LIST(LogicalType::BIGINT)}, + LogicalType::LIST(LogicalTypeId::ANY), ListSelectFunction, + ListSelectBind); + return fun; +} + +void ListSelectFun::RegisterFunction(BuiltinFunctions &set) { + set.AddFunction({"list_select", "array_select"}, ListSelectFun::GetFunction()); + set.AddFunction({"list_where", "array_where"}, ListWhereFun::GetFunction()); +} +} // namespace duckdb diff --git a/src/duckdb/src/function/scalar/list/list_zip.cpp b/src/duckdb/src/function/scalar/list/list_zip.cpp new file mode 100644 index 000000000..ab5548321 --- /dev/null +++ b/src/duckdb/src/function/scalar/list/list_zip.cpp @@ -0,0 +1,164 @@ +#include "duckdb/common/types/data_chunk.hpp" +#include "duckdb/function/scalar/nested_functions.hpp" +#include "duckdb/planner/expression_binder.hpp" +#include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/planner/expression/bound_parameter_expression.hpp" +#include "duckdb/common/to_string.hpp" + +namespace duckdb { + +static void ListZipFunction(DataChunk &args, ExpressionState &state, Vector &result) { + idx_t count = args.size(); + idx_t args_size = args.ColumnCount(); + auto *result_data = FlatVector::GetData(result); + auto &result_struct = ListVector::GetEntry(result); + auto &struct_entries = StructVector::GetEntries(result_struct); + bool truncate_flags_set = false; + + // Check flag + if (args.data.back().GetType().id() == LogicalTypeId::BOOLEAN) { + truncate_flags_set = true; + args_size--; + } + + vector input_lists; + for (idx_t i = 0; i < args.ColumnCount(); i++) { + UnifiedVectorFormat curr; + args.data[i].ToUnifiedFormat(count, curr); + input_lists.push_back(curr); + } + + // Handling output row for each input row + idx_t result_size = 0; + vector lengths; + for (idx_t j = 0; j < count; j++) { + + // Is flag for current row set + bool truncate_to_shortest = false; + if (truncate_flags_set) { + idx_t flag_idx = input_lists.back().sel->get_index(j); + auto flag_data = UnifiedVectorFormat::GetData(input_lists.back()); + truncate_to_shortest = flag_data[flag_idx]; + } + + // Calculation of the outgoing list size + idx_t len = truncate_to_shortest ? NumericLimits::Maximum() : 0; + for (idx_t i = 0; i < args_size; i++) { + idx_t curr_size; + if (args.data[i].GetType() == LogicalType::SQLNULL || ListVector::GetListSize(args.data[i]) == 0) { + curr_size = 0; + } else { + idx_t sel_idx = input_lists[i].sel->get_index(j); + auto curr_data = UnifiedVectorFormat::GetData(input_lists[i]); + curr_size = input_lists[i].validity.RowIsValid(sel_idx) ? curr_data[sel_idx].length : 0; + } + + // Dependent on flag using gt or lt + if (truncate_to_shortest) { + len = len > curr_size ? curr_size : len; + } else { + len = len < curr_size ? curr_size : len; + } + } + lengths.push_back(len); + result_size += len; + } + + ListVector::SetListSize(result, result_size); + ListVector::Reserve(result, result_size); + vector selections; + vector masks; + for (idx_t i = 0; i < args_size; i++) { + selections.push_back(SelectionVector(result_size)); + masks.push_back(ValidityMask(result_size)); + } + + idx_t offset = 0; + for (idx_t j = 0; j < count; j++) { + idx_t len = lengths[j]; + for (idx_t i = 0; i < args_size; i++) { + UnifiedVectorFormat curr = input_lists[i]; + idx_t sel_idx = curr.sel->get_index(j); + idx_t curr_off = 0; + idx_t curr_len = 0; + + // Copying values from the given lists + if (curr.validity.RowIsValid(sel_idx)) { + auto input_lists_data = UnifiedVectorFormat::GetData(curr); + curr_off = input_lists_data[sel_idx].offset; + curr_len = input_lists_data[sel_idx].length; + auto copy_len = len < curr_len ? len : curr_len; + idx_t entry = offset; + for (idx_t k = 0; k < copy_len; k++) { + if (!FlatVector::Validity(ListVector::GetEntry(args.data[i])).RowIsValid(curr_off + k)) { + masks[i].SetInvalid(entry + k); + } + selections[i].set_index(entry + k, curr_off + k); + } + } + + // Set NULL values for list that are shorter than the output list + if (len > curr_len) { + for (idx_t d = curr_len; d < len; d++) { + masks[i].SetInvalid(d + offset); + selections[i].set_index(d + offset, 0); + } + } + } + result_data[j].length = len; + result_data[j].offset = offset; + offset += len; + } + for (idx_t child_idx = 0; child_idx < args_size; child_idx++) { + if (!(args.data[child_idx].GetType() == LogicalType::SQLNULL)) { + struct_entries[child_idx]->Slice(ListVector::GetEntry(args.data[child_idx]), selections[child_idx], + result_size); + } + struct_entries[child_idx]->Flatten(result_size); + FlatVector::SetValidity((*struct_entries[child_idx]), masks[child_idx]); + } + result.SetVectorType(args.AllConstant() ? VectorType::CONSTANT_VECTOR : VectorType::FLAT_VECTOR); +} + +static unique_ptr ListZipBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + child_list_t struct_children; + + // The last argument could be a flag to be set if we want a minimal list or a maximal list + idx_t size = arguments.size(); + if (arguments[size - 1]->return_type.id() == LogicalTypeId::BOOLEAN) { + size--; + } + + for (idx_t i = 0; i < size; i++) { + auto &child = arguments[i]; + if (child->alias.empty()) { + child->alias = "list_" + to_string(i + 1); + } + switch (child->return_type.id()) { + case LogicalTypeId::LIST: + struct_children.push_back(make_pair(child->alias, ListType::GetChildType(child->return_type))); + break; + case LogicalTypeId::SQLNULL: + struct_children.push_back(make_pair(child->alias, LogicalTypeId::SQLNULL)); + break; + default: + throw ParameterNotResolvedException(); + } + } + bound_function.return_type = LogicalType::LIST(LogicalType::STRUCT(struct_children)); + return make_uniq(bound_function.return_type); +} + +ScalarFunction ListZipFun::GetFunction() { + + auto fun = ScalarFunction({}, LogicalType::LIST(LogicalTypeId::STRUCT), ListZipFunction, ListZipBind); + fun.varargs = LogicalType::ANY; + fun.null_handling = FunctionNullHandling::SPECIAL_HANDLING; // Special handling needed? + return fun; +} + +void ListZipFun::RegisterFunction(BuiltinFunctions &set) { + set.AddFunction({"list_zip", "array_zip"}, GetFunction()); +} +} // namespace duckdb diff --git a/src/duckdb/src/function/scalar/nested_functions.cpp b/src/duckdb/src/function/scalar/nested_functions.cpp index fa05ac7dc..a0e0aa5a9 100644 --- a/src/duckdb/src/function/scalar/nested_functions.cpp +++ b/src/duckdb/src/function/scalar/nested_functions.cpp @@ -9,6 +9,8 @@ void BuiltinFunctions::RegisterNestedFunctions() { Register(); Register(); Register(); + Register(); + Register(); } } // namespace duckdb diff --git a/src/duckdb/src/function/scalar/string/length.cpp b/src/duckdb/src/function/scalar/string/length.cpp index d4fe021be..218e4e846 100644 --- a/src/duckdb/src/function/scalar/string/length.cpp +++ b/src/duckdb/src/function/scalar/string/length.cpp @@ -25,23 +25,6 @@ struct GraphemeCountOperator { } }; -struct ArrayLengthOperator { - template - static inline TR Operation(TA input) { - return input.length; - } -}; - -struct ArrayLengthBinaryOperator { - template - static inline TR Operation(TA input, TB dimension) { - if (dimension != 1) { - throw NotImplementedException("array_length for dimensions other than 1 not implemented"); - } - return input.length; - } -}; - // strlen returns the size in bytes struct StrLenOperator { template @@ -84,19 +67,137 @@ static unique_ptr LengthPropagateStats(ClientContext &context, F return nullptr; } -static unique_ptr ListLengthBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments) { +//------------------------------------------------------------------ +// ARRAY / LIST LENGTH +//------------------------------------------------------------------ + +static void ListLengthFunction(DataChunk &args, ExpressionState &state, Vector &result) { + auto &input = args.data[0]; + D_ASSERT(input.GetType().id() == LogicalTypeId::LIST); + UnaryExecutor::Execute(input, result, args.size(), + [](list_entry_t input) { return input.length; }); + if (args.AllConstant()) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +static void ArrayLengthFunction(DataChunk &args, ExpressionState &state, Vector &result) { + auto &input = args.data[0]; + // If the input is an array, the length is constant + result.SetVectorType(VectorType::CONSTANT_VECTOR); + ConstantVector::GetData(result)[0] = static_cast(ArrayType::GetSize(input.GetType())); +} + +static unique_ptr ArrayOrListLengthBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { if (arguments[0]->HasParameter()) { throw ParameterNotResolvedException(); } + if (arguments[0]->return_type.id() == LogicalTypeId::ARRAY) { + bound_function.function = ArrayLengthFunction; + } else if (arguments[0]->return_type.id() == LogicalTypeId::LIST) { + bound_function.function = ListLengthFunction; + } else { + // Unreachable + throw BinderException("length can only be used on arrays or lists"); + } bound_function.arguments[0] = arguments[0]->return_type; return nullptr; } +//------------------------------------------------------------------ +// ARRAY / LIST WITH DIMENSION +//------------------------------------------------------------------ +static void ListLengthBinaryFunction(DataChunk &args, ExpressionState &, Vector &result) { + auto type = args.data[0].GetType(); + auto &input = args.data[0]; + auto &dimension = args.data[1]; + BinaryExecutor::Execute( + input, dimension, result, args.size(), [](list_entry_t input, int64_t dimension) { + if (dimension != 1) { + throw NotImplementedException("array_length for lists with dimensions other than 1 not implemented"); + } + return input.length; + }); + if (args.AllConstant()) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +struct ArrayLengthBinaryFunctionData : public FunctionData { + vector dimensions; + + unique_ptr Copy() const override { + auto copy = make_uniq(); + copy->dimensions = dimensions; + return std::move(copy); + } + + bool Equals(const FunctionData &other) const override { + auto &other_data = other.Cast(); + return dimensions == other_data.dimensions; + } +}; + +static void ArrayLengthBinaryFunction(DataChunk &args, ExpressionState &state, Vector &result) { + auto type = args.data[0].GetType(); + auto &dimension = args.data[1]; + + auto &expr = state.expr.Cast(); + auto &data = expr.bind_info->Cast(); + auto &dimensions = data.dimensions; + auto max_dimension = static_cast(dimensions.size()); + + UnaryExecutor::Execute(dimension, result, args.size(), [&](int64_t dimension) { + if (dimension < 1 || dimension > max_dimension) { + throw OutOfRangeException(StringUtil::Format( + "array_length dimension '%lld' out of range (min: '1', max: '%lld')", dimension, max_dimension)); + } + return dimensions[dimension - 1]; + }); + + if (args.AllConstant()) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +static unique_ptr ArrayOrListLengthBinaryBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + if (arguments[0]->HasParameter()) { + throw ParameterNotResolvedException(); + } + auto type = arguments[0]->return_type; + if (type.id() == LogicalTypeId::ARRAY) { + bound_function.arguments[0] = type; + bound_function.function = ArrayLengthBinaryFunction; + + // If the input is an array, the dimensions are constant, so we can calculate them at bind time + vector dimensions; + while (true) { + if (type.id() == LogicalTypeId::ARRAY) { + dimensions.push_back(ArrayType::GetSize(type)); + type = ArrayType::GetChildType(type); + } else { + break; + } + } + auto data = make_uniq(); + data->dimensions = dimensions; + return std::move(data); + + } else if (type.id() == LogicalTypeId::LIST) { + bound_function.function = ListLengthBinaryFunction; + bound_function.arguments[0] = type; + return nullptr; + } else { + // Unreachable + throw BinderException("array_length can only be used on arrays or lists"); + } +} + void LengthFun::RegisterFunction(BuiltinFunctions &set) { ScalarFunction array_length_unary = - ScalarFunction({LogicalType::LIST(LogicalType::ANY)}, LogicalType::BIGINT, - ScalarFunction::UnaryFunction, ListLengthBind); + ScalarFunction({LogicalType::LIST(LogicalType::ANY)}, LogicalType::BIGINT, nullptr, ArrayOrListLengthBind); ScalarFunctionSet length("length"); length.AddFunction(ScalarFunction({LogicalType::VARCHAR}, LogicalType::BIGINT, ScalarFunction::UnaryFunction, nullptr, @@ -116,9 +217,8 @@ void LengthFun::RegisterFunction(BuiltinFunctions &set) { ScalarFunctionSet array_length("array_length"); array_length.AddFunction(array_length_unary); - array_length.AddFunction(ScalarFunction( - {LogicalType::LIST(LogicalType::ANY), LogicalType::BIGINT}, LogicalType::BIGINT, - ScalarFunction::BinaryFunction, ListLengthBind)); + array_length.AddFunction(ScalarFunction({LogicalType::LIST(LogicalType::ANY), LogicalType::BIGINT}, + LogicalType::BIGINT, nullptr, ArrayOrListLengthBinaryBind)); set.AddFunction(array_length); set.AddFunction(ScalarFunction("strlen", {LogicalType::VARCHAR}, LogicalType::BIGINT, diff --git a/src/duckdb/src/function/scalar_function.cpp b/src/duckdb/src/function/scalar_function.cpp index 250eee74a..ac7b80dba 100644 --- a/src/duckdb/src/function/scalar_function.cpp +++ b/src/duckdb/src/function/scalar_function.cpp @@ -9,25 +9,27 @@ ScalarFunction::ScalarFunction(string name, vector arguments, Logic scalar_function_t function, bind_scalar_function_t bind, dependency_function_t dependency, function_statistics_t statistics, init_local_state_t init_local_state, LogicalType varargs, - FunctionSideEffects side_effects, FunctionNullHandling null_handling) + FunctionSideEffects side_effects, FunctionNullHandling null_handling, + bind_lambda_function_t bind_lambda) : BaseScalarFunction(std::move(name), std::move(arguments), std::move(return_type), side_effects, std::move(varargs), null_handling), function(std::move(function)), bind(bind), init_local_state(init_local_state), dependency(dependency), - statistics(statistics), serialize(nullptr), deserialize(nullptr) { + statistics(statistics), bind_lambda(bind_lambda), serialize(nullptr), deserialize(nullptr) { } ScalarFunction::ScalarFunction(vector arguments, LogicalType return_type, scalar_function_t function, bind_scalar_function_t bind, dependency_function_t dependency, function_statistics_t statistics, init_local_state_t init_local_state, LogicalType varargs, FunctionSideEffects side_effects, - FunctionNullHandling null_handling) + FunctionNullHandling null_handling, bind_lambda_function_t bind_lambda) : ScalarFunction(string(), std::move(arguments), std::move(return_type), std::move(function), bind, dependency, - statistics, init_local_state, std::move(varargs), side_effects, null_handling) { + statistics, init_local_state, std::move(varargs), side_effects, null_handling, bind_lambda) { } bool ScalarFunction::operator==(const ScalarFunction &rhs) const { return name == rhs.name && arguments == rhs.arguments && return_type == rhs.return_type && varargs == rhs.varargs && - bind == rhs.bind && dependency == rhs.dependency && statistics == rhs.statistics; + bind == rhs.bind && dependency == rhs.dependency && statistics == rhs.statistics && + bind_lambda == rhs.bind_lambda; } bool ScalarFunction::operator!=(const ScalarFunction &rhs) const { diff --git a/src/duckdb/src/function/table/arrow_conversion.cpp b/src/duckdb/src/function/table/arrow_conversion.cpp index 7aaede6f5..204078357 100644 --- a/src/duckdb/src/function/table/arrow_conversion.cpp +++ b/src/duckdb/src/function/table/arrow_conversion.cpp @@ -18,26 +18,13 @@ static void ShiftRight(unsigned char *ar, int size, int shift) { } } -idx_t GetEffectiveOffset(ArrowArray &array, int64_t parent_offset, ArrowScanLocalState &state, - int64_t nested_offset = -1) { - if (nested_offset != -1) { - // The parent of this array is a list - // We just ignore the parent offset, it's already applied to the list - return array.offset + nested_offset; - } - // Parent offset is set in the case of a struct, it applies to all child arrays - // 'chunk_offset' is how much of the chunk we've already scanned, in case the chunk size exceeds - // STANDARD_VECTOR_SIZE - return array.offset + parent_offset + state.chunk_offset; -} - template T *ArrowBufferData(ArrowArray &array, idx_t buffer_idx) { return (T *)array.buffers[buffer_idx]; // NOLINT } static void GetValidityMask(ValidityMask &mask, ArrowArray &array, ArrowScanLocalState &scan_state, idx_t size, - int64_t parent_offset, int64_t nested_offset = -1, bool add_null = false) { + int64_t nested_offset = -1, bool add_null = false) { // In certains we don't need to or cannot copy arrow's validity mask to duckdb. // // The conditions where we do want to copy arrow's mask to duckdb are: @@ -45,7 +32,10 @@ static void GetValidityMask(ValidityMask &mask, ArrowArray &array, ArrowScanLoca // 2. n_buffers > 0, meaning the array's arrow type is not `null` // 3. the validity buffer (the first buffer) is not a nullptr if (array.null_count != 0 && array.n_buffers > 0 && array.buffers[0]) { - auto bit_offset = GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto bit_offset = scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + bit_offset = nested_offset; + } mask.EnsureWritable(); #if STANDARD_VECTOR_SIZE > 64 auto n_bitmask_bytes = (size + 8 - 1) / 8; @@ -84,10 +74,10 @@ static void GetValidityMask(ValidityMask &mask, ArrowArray &array, ArrowScanLoca } static void SetValidityMask(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, idx_t size, - int64_t parent_offset, int64_t nested_offset, bool add_null = false) { + int64_t nested_offset, bool add_null = false) { D_ASSERT(vector.GetVectorType() == VectorType::FLAT_VECTOR); auto &mask = FlatVector::Validity(vector); - GetValidityMask(mask, array, scan_state, size, parent_offset, nested_offset, add_null); + GetValidityMask(mask, array, scan_state, size, nested_offset, add_null); } static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, @@ -99,19 +89,21 @@ static void ColumnArrowToDuckDBDictionary(Vector &vector, ArrowArray &array, Arr ValidityMask *parent_mask = nullptr, uint64_t parent_offset = 0); static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, - const ArrowType &arrow_type, int64_t nested_offset, ValidityMask *parent_mask, - int64_t parent_offset) { + const ArrowType &arrow_type, int64_t nested_offset, ValidityMask *parent_mask) { auto size_type = arrow_type.GetSizeType(); idx_t list_size = 0; auto &scan_state = array_state.state; - SetValidityMask(vector, array, scan_state, size, parent_offset, nested_offset); + SetValidityMask(vector, array, scan_state, size, nested_offset); idx_t start_offset = 0; idx_t cur_offset = 0; if (size_type == ArrowVariableSizeType::FIXED_SIZE) { auto fixed_size = arrow_type.FixedSize(); //! Have to check validity mask before setting this up - idx_t offset = GetEffectiveOffset(array, parent_offset, scan_state, nested_offset) * fixed_size; + idx_t offset = (scan_state.chunk_offset + array.offset) * fixed_size; + if (nested_offset != -1) { + offset = fixed_size * nested_offset; + } start_offset = offset; auto list_data = FlatVector::GetData(vector); for (idx_t i = 0; i < size; i++) { @@ -122,8 +114,10 @@ static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanS } list_size = start_offset + cur_offset; } else if (size_type == ArrowVariableSizeType::NORMAL) { - auto offsets = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto offsets = ArrowBufferData(array, 1) + array.offset + scan_state.chunk_offset; + if (nested_offset != -1) { + offsets = ArrowBufferData(array, 1) + nested_offset; + } start_offset = offsets[0]; auto list_data = FlatVector::GetData(vector); for (idx_t i = 0; i < size; i++) { @@ -134,8 +128,10 @@ static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanS } list_size = offsets[size]; } else { - auto offsets = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto offsets = ArrowBufferData(array, 1) + array.offset + scan_state.chunk_offset; + if (nested_offset != -1) { + offsets = ArrowBufferData(array, 1) + nested_offset; + } start_offset = offsets[0]; auto list_data = FlatVector::GetData(vector); for (idx_t i = 0; i < size; i++) { @@ -150,7 +146,7 @@ static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanS ListVector::Reserve(vector, list_size); ListVector::SetListSize(vector, list_size); auto &child_vector = ListVector::GetEntry(vector); - SetValidityMask(child_vector, *array.children[0], scan_state, list_size, array.offset, start_offset); + SetValidityMask(child_vector, *array.children[0], scan_state, list_size, start_offset); auto &list_mask = FlatVector::Validity(vector); if (parent_mask) { //! Since this List is owned by a struct we must guarantee their validity map matches on Null @@ -179,13 +175,16 @@ static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanS } static void ArrowToDuckDBBlob(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, idx_t size, - const ArrowType &arrow_type, int64_t nested_offset, int64_t parent_offset) { + const ArrowType &arrow_type, int64_t nested_offset) { auto size_type = arrow_type.GetSizeType(); - SetValidityMask(vector, array, scan_state, size, parent_offset, nested_offset); + SetValidityMask(vector, array, scan_state, size, nested_offset); if (size_type == ArrowVariableSizeType::FIXED_SIZE) { auto fixed_size = arrow_type.FixedSize(); //! Have to check validity mask before setting this up - idx_t offset = GetEffectiveOffset(array, parent_offset, scan_state, nested_offset) * fixed_size; + idx_t offset = (scan_state.chunk_offset + array.offset) * fixed_size; + if (nested_offset != -1) { + offset = fixed_size * nested_offset; + } auto cdata = ArrowBufferData(array, 1); for (idx_t row_idx = 0; row_idx < size; row_idx++) { if (FlatVector::IsNull(vector, row_idx)) { @@ -197,8 +196,10 @@ static void ArrowToDuckDBBlob(Vector &vector, ArrowArray &array, ArrowScanLocalS offset += blob_len; } } else if (size_type == ArrowVariableSizeType::NORMAL) { - auto offsets = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto offsets = ArrowBufferData(array, 1) + array.offset + scan_state.chunk_offset; + if (nested_offset != -1) { + offsets = ArrowBufferData(array, 1) + array.offset + nested_offset; + } auto cdata = ArrowBufferData(array, 2); for (idx_t row_idx = 0; row_idx < size; row_idx++) { if (FlatVector::IsNull(vector, row_idx)) { @@ -213,8 +214,10 @@ static void ArrowToDuckDBBlob(Vector &vector, ArrowArray &array, ArrowScanLocalS if (ArrowBufferData(array, 1)[array.length] > NumericLimits::Maximum()) { // LCOV_EXCL_START throw ConversionException("DuckDB does not support Blobs over 4GB"); } // LCOV_EXCL_STOP - auto offsets = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto offsets = ArrowBufferData(array, 1) + array.offset + scan_state.chunk_offset; + if (nested_offset != -1) { + offsets = ArrowBufferData(array, 1) + array.offset + nested_offset; + } auto cdata = ArrowBufferData(array, 2); for (idx_t row_idx = 0; row_idx < size; row_idx++) { if (FlatVector::IsNull(vector, row_idx)) { @@ -266,17 +269,23 @@ static void SetVectorString(Vector &vector, idx_t size, char *cdata, T *offsets) static void DirectConversion(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, int64_t nested_offset, uint64_t parent_offset) { auto internal_type = GetTypeIdSize(vector.GetType().InternalType()); - auto data_ptr = ArrowBufferData(array, 1) + - internal_type * GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto data_ptr = + ArrowBufferData(array, 1) + internal_type * (scan_state.chunk_offset + array.offset + parent_offset); + if (nested_offset != -1) { + data_ptr = ArrowBufferData(array, 1) + internal_type * (array.offset + nested_offset + parent_offset); + } FlatVector::SetData(vector, data_ptr); } template static void TimeConversion(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, int64_t nested_offset, - int64_t parent_offset, idx_t size, int64_t conversion) { + idx_t size, int64_t conversion) { auto tgt_ptr = FlatVector::GetData(vector); auto &validity_mask = FlatVector::Validity(vector); - auto src_ptr = (T *)array.buffers[1] + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = (T *)array.buffers[1] + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = (T *)array.buffers[1] + nested_offset + array.offset; + } for (idx_t row = 0; row < size; row++) { if (!validity_mask.RowIsValid(row)) { continue; @@ -288,11 +297,13 @@ static void TimeConversion(Vector &vector, ArrowArray &array, ArrowScanLocalStat } static void TimestampTZConversion(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size, int64_t conversion) { + int64_t nested_offset, idx_t size, int64_t conversion) { auto tgt_ptr = FlatVector::GetData(vector); auto &validity_mask = FlatVector::Validity(vector); - auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } for (idx_t row = 0; row < size; row++) { if (!validity_mask.RowIsValid(row)) { continue; @@ -304,10 +315,12 @@ static void TimestampTZConversion(Vector &vector, ArrowArray &array, ArrowScanLo } static void IntervalConversionUs(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size, int64_t conversion) { + int64_t nested_offset, idx_t size, int64_t conversion) { auto tgt_ptr = FlatVector::GetData(vector); - auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } for (idx_t row = 0; row < size; row++) { tgt_ptr[row].days = 0; tgt_ptr[row].months = 0; @@ -318,10 +331,12 @@ static void IntervalConversionUs(Vector &vector, ArrowArray &array, ArrowScanLoc } static void IntervalConversionMonths(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size) { + int64_t nested_offset, idx_t size) { auto tgt_ptr = FlatVector::GetData(vector); - auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } for (idx_t row = 0; row < size; row++) { tgt_ptr[row].days = 0; tgt_ptr[row].micros = 0; @@ -330,10 +345,12 @@ static void IntervalConversionMonths(Vector &vector, ArrowArray &array, ArrowSca } static void IntervalConversionMonthDayNanos(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size) { + int64_t nested_offset, idx_t size) { auto tgt_ptr = FlatVector::GetData(vector); - auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } for (idx_t row = 0; row < size; row++) { tgt_ptr[row].days = src_ptr[row].days; tgt_ptr[row].micros = src_ptr[row].nanoseconds / Interval::NANOS_PER_MICRO; @@ -344,9 +361,6 @@ static void IntervalConversionMonthDayNanos(Vector &vector, ArrowArray &array, A static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, const ArrowType &arrow_type, int64_t nested_offset, ValidityMask *parent_mask, uint64_t parent_offset) { - if (parent_offset != 0) { - (void)array_state; - } auto &scan_state = array_state.state; D_ASSERT(!array.dictionary); switch (vector.GetType().id()) { @@ -356,8 +370,11 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca case LogicalTypeId::BOOLEAN: { //! Arrow bit-packs boolean values //! Lets first figure out where we are in the source array - auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset) / 8; + auto src_ptr = ArrowBufferData(array, 1) + (scan_state.chunk_offset + array.offset) / 8; + + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + (nested_offset + array.offset) / 8; + } auto tgt_ptr = (uint8_t *)FlatVector::GetData(vector); int src_pos = 0; idx_t cur_bit = scan_state.chunk_offset % 8; @@ -400,12 +417,16 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto size_type = arrow_type.GetSizeType(); auto cdata = ArrowBufferData(array, 2); if (size_type == ArrowVariableSizeType::SUPER_SIZE) { - auto offsets = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto offsets = ArrowBufferData(array, 1) + array.offset + scan_state.chunk_offset; + if (nested_offset != -1) { + offsets = ArrowBufferData(array, 1) + array.offset + nested_offset; + } SetVectorString(vector, size, cdata, offsets); } else { - auto offsets = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto offsets = ArrowBufferData(array, 1) + array.offset + scan_state.chunk_offset; + if (nested_offset != -1) { + offsets = ArrowBufferData(array, 1) + array.offset + nested_offset; + } SetVectorString(vector, size, cdata, offsets); } break; @@ -420,8 +441,10 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca } case ArrowDateTimeType::MILLISECONDS: { //! convert date from nanoseconds to days - auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } auto tgt_ptr = FlatVector::GetData(vector); for (idx_t row = 0; row < size; row++) { tgt_ptr[row] = date_t(int64_t(src_ptr[row]) / static_cast(1000 * 60 * 60 * 24)); @@ -437,21 +460,23 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto precision = arrow_type.GetDateTimeType(); switch (precision) { case ArrowDateTimeType::SECONDS: { - TimeConversion(vector, array, scan_state, nested_offset, parent_offset, size, 1000000); + TimeConversion(vector, array, scan_state, nested_offset, size, 1000000); break; } case ArrowDateTimeType::MILLISECONDS: { - TimeConversion(vector, array, scan_state, nested_offset, parent_offset, size, 1000); + TimeConversion(vector, array, scan_state, nested_offset, size, 1000); break; } case ArrowDateTimeType::MICROSECONDS: { - TimeConversion(vector, array, scan_state, nested_offset, parent_offset, size, 1); + TimeConversion(vector, array, scan_state, nested_offset, size, 1); break; } case ArrowDateTimeType::NANOSECONDS: { auto tgt_ptr = FlatVector::GetData(vector); - auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } for (idx_t row = 0; row < size; row++) { tgt_ptr[row].micros = src_ptr[row] / 1000; } @@ -466,11 +491,11 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto precision = arrow_type.GetDateTimeType(); switch (precision) { case ArrowDateTimeType::SECONDS: { - TimestampTZConversion(vector, array, scan_state, nested_offset, parent_offset, size, 1000000); + TimestampTZConversion(vector, array, scan_state, nested_offset, size, 1000000); break; } case ArrowDateTimeType::MILLISECONDS: { - TimestampTZConversion(vector, array, scan_state, nested_offset, parent_offset, size, 1000); + TimestampTZConversion(vector, array, scan_state, nested_offset, size, 1000); break; } case ArrowDateTimeType::MICROSECONDS: { @@ -479,8 +504,10 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca } case ArrowDateTimeType::NANOSECONDS: { auto tgt_ptr = FlatVector::GetData(vector); - auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } for (idx_t row = 0; row < size; row++) { tgt_ptr[row].value = src_ptr[row] / 1000; } @@ -495,22 +522,24 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto precision = arrow_type.GetDateTimeType(); switch (precision) { case ArrowDateTimeType::SECONDS: { - IntervalConversionUs(vector, array, scan_state, nested_offset, parent_offset, size, 1000000); + IntervalConversionUs(vector, array, scan_state, nested_offset, size, 1000000); break; } case ArrowDateTimeType::DAYS: case ArrowDateTimeType::MILLISECONDS: { - IntervalConversionUs(vector, array, scan_state, nested_offset, parent_offset, size, 1000); + IntervalConversionUs(vector, array, scan_state, nested_offset, size, 1000); break; } case ArrowDateTimeType::MICROSECONDS: { - IntervalConversionUs(vector, array, scan_state, nested_offset, parent_offset, size, 1); + IntervalConversionUs(vector, array, scan_state, nested_offset, size, 1); break; } case ArrowDateTimeType::NANOSECONDS: { auto tgt_ptr = FlatVector::GetData(vector); - auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } for (idx_t row = 0; row < size; row++) { tgt_ptr[row].micros = src_ptr[row] / 1000; tgt_ptr[row].days = 0; @@ -519,11 +548,11 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca break; } case ArrowDateTimeType::MONTHS: { - IntervalConversionMonths(vector, array, scan_state, nested_offset, parent_offset, size); + IntervalConversionMonths(vector, array, scan_state, nested_offset, size); break; } case ArrowDateTimeType::MONTH_DAY_NANO: { - IntervalConversionMonthDayNanos(vector, array, scan_state, nested_offset, parent_offset, size); + IntervalConversionMonthDayNanos(vector, array, scan_state, nested_offset, size); break; } default: @@ -534,8 +563,10 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca case LogicalTypeId::DECIMAL: { auto val_mask = FlatVector::Validity(vector); //! We have to convert from INT128 - auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + scan_state.chunk_offset + array.offset; + if (nested_offset != -1) { + src_ptr = ArrowBufferData(array, 1) + nested_offset + array.offset; + } switch (vector.GetType().InternalType()) { case PhysicalType::INT16: { auto tgt_ptr = FlatVector::GetData(vector); @@ -571,9 +602,9 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca break; } case PhysicalType::INT128: { - FlatVector::SetData(vector, ArrowBufferData(array, 1) + - GetTypeIdSize(vector.GetType().InternalType()) * - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset)); + FlatVector::SetData(vector, + ArrowBufferData(array, 1) + GetTypeIdSize(vector.GetType().InternalType()) * + (scan_state.chunk_offset + array.offset)); break; } default: @@ -583,15 +614,15 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca break; } case LogicalTypeId::BLOB: { - ArrowToDuckDBBlob(vector, array, scan_state, size, arrow_type, nested_offset, parent_offset); + ArrowToDuckDBBlob(vector, array, scan_state, size, arrow_type, nested_offset); break; } case LogicalTypeId::LIST: { - ArrowToDuckDBList(vector, array, array_state, size, arrow_type, nested_offset, parent_mask, parent_offset); + ArrowToDuckDBList(vector, array, array_state, size, arrow_type, nested_offset, parent_mask); break; } case LogicalTypeId::MAP: { - ArrowToDuckDBList(vector, array, array_state, size, arrow_type, nested_offset, parent_mask, parent_offset); + ArrowToDuckDBList(vector, array, array_state, size, arrow_type, nested_offset, parent_mask); ArrowToDuckDBMapVerify(vector, size); break; } @@ -605,7 +636,7 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto &child_type = arrow_type[child_idx]; auto &child_state = array_state.GetChild(child_idx); - SetValidityMask(child_entry, child_array, scan_state, size, array.offset, nested_offset); + SetValidityMask(child_entry, child_array, scan_state, size, nested_offset); if (!struct_validity_mask.AllValid()) { auto &child_validity_mark = FlatVector::Validity(child_entry); for (idx_t i = 0; i < size; i++) { @@ -615,6 +646,7 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca } } if (child_array.dictionary) { + // TODO: add support for offsets ColumnArrowToDuckDBDictionary(child_entry, child_array, child_state, size, child_type, nested_offset, &struct_validity_mask, array.offset); } else { @@ -638,7 +670,7 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto &child_state = array_state.GetChild(child_idx); auto &child_type = arrow_type[child_idx]; - SetValidityMask(child, child_array, scan_state, size, parent_offset, nested_offset); + SetValidityMask(child, child_array, scan_state, size, nested_offset); if (child_array.dictionary) { ColumnArrowToDuckDBDictionary(child, child_array, child_state, size, child_type); @@ -791,48 +823,26 @@ static void SetSelectionVector(SelectionVector &sel, data_ptr_t indices_p, Logic } } -static bool CanContainNull(ArrowArray &array, ValidityMask *parent_mask) { - if (array.null_count > 0) { - return true; - } - if (!parent_mask) { - return false; - } - return !parent_mask->AllValid(); -} - static void ColumnArrowToDuckDBDictionary(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, const ArrowType &arrow_type, int64_t nested_offset, ValidityMask *parent_mask, uint64_t parent_offset) { + SelectionVector sel; auto &scan_state = array_state.state; - - const bool has_nulls = CanContainNull(array, parent_mask); if (!array_state.HasDictionary()) { //! We need to set the dictionary data for this column auto base_vector = make_uniq(vector.GetType(), array.dictionary->length); - SetValidityMask(*base_vector, *array.dictionary, scan_state, array.dictionary->length, 0, 0, has_nulls); + SetValidityMask(*base_vector, *array.dictionary, scan_state, array.dictionary->length, 0, array.null_count > 0); ColumnArrowToDuckDB(*base_vector, *array.dictionary, array_state, array.dictionary->length, arrow_type.GetDictionary()); array_state.AddDictionary(std::move(base_vector)); } auto offset_type = arrow_type.GetDuckType(); //! Get Pointer to Indices of Dictionary - auto indices = - ArrowBufferData(array, 1) + - GetTypeIdSize(offset_type.InternalType()) * GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); - - SelectionVector sel; - if (has_nulls) { + auto indices = ArrowBufferData(array, 1) + + GetTypeIdSize(offset_type.InternalType()) * (scan_state.chunk_offset + array.offset); + if (array.null_count > 0) { ValidityMask indices_validity; - GetValidityMask(indices_validity, array, scan_state, size, parent_offset); - if (parent_mask && !parent_mask->AllValid()) { - auto &struct_validity_mask = *parent_mask; - for (idx_t i = 0; i < size; i++) { - if (!struct_validity_mask.RowIsValid(i)) { - indices_validity.SetInvalid(i); - } - } - } + GetValidityMask(indices_validity, array, scan_state, size); SetSelectionVector(sel, indices, offset_type, size, &indices_validity, array.dictionary->length); } else { SetSelectionVector(sel, indices, offset_type, size); @@ -854,7 +864,6 @@ void ArrowTableFunction::ArrowToDuckDB(ArrowScanLocalState &scan_state, const ar continue; } - auto &parent_array = scan_state.chunk->arrow_array; auto &array = *scan_state.chunk->arrow_array.children[arrow_array_idx]; if (!array.release) { throw InvalidInputException("arrow_scan: released array passed"); @@ -879,7 +888,7 @@ void ArrowTableFunction::ArrowToDuckDB(ArrowScanLocalState &scan_state, const ar if (array.dictionary) { ColumnArrowToDuckDBDictionary(output.data[idx], array, array_state, output.size(), arrow_type); } else { - SetValidityMask(output.data[idx], array, scan_state, output.size(), parent_array.offset, -1); + SetValidityMask(output.data[idx], array, scan_state, output.size(), -1); ColumnArrowToDuckDB(output.data[idx], array, array_state, output.size(), arrow_type); } } diff --git a/src/duckdb/src/function/table/copy_csv.cpp b/src/duckdb/src/function/table/copy_csv.cpp index e721fe465..fcb5957d0 100644 --- a/src/duckdb/src/function/table/copy_csv.cpp +++ b/src/duckdb/src/function/table/copy_csv.cpp @@ -159,15 +159,11 @@ static unique_ptr ReadCSVBind(ClientContext &context, CopyInfo &in bind_data->FinalizeRead(context); if (options.auto_detect) { - // We must run the sniffer. + // We must run the sniffer, but this is a copy csv, hence names and types have already been previsouly defined. auto file_handle = BaseCSVReader::OpenCSV(context, options); auto buffer_manager = make_shared(context, std::move(file_handle), options); - CSVSniffer sniffer(options, buffer_manager, bind_data->state_machine_cache); - auto sniffer_result = sniffer.SniffCSV(); - bind_data->csv_types = sniffer_result.return_types; - bind_data->csv_names = sniffer_result.names; - bind_data->return_types = sniffer_result.return_types; - bind_data->return_names = sniffer_result.names; + CSVSniffer sniffer(options, buffer_manager, bind_data->state_machine_cache, {&expected_types, &expected_names}); + sniffer.SniffCSV(); } return std::move(bind_data); } diff --git a/src/duckdb/src/function/table/read_csv.cpp b/src/duckdb/src/function/table/read_csv.cpp index bd5baea3e..9cc864cb9 100644 --- a/src/duckdb/src/function/table/read_csv.cpp +++ b/src/duckdb/src/function/table/read_csv.cpp @@ -47,21 +47,7 @@ void ReadCSVData::FinalizeRead(ClientContext &context) { single_threaded = true; } - // Validate rejects_table options - if (!options.rejects_table_name.empty()) { - if (!options.ignore_errors) { - throw BinderException("REJECTS_TABLE option is only supported when IGNORE_ERRORS is set to true"); - } - if (options.file_options.union_by_name) { - throw BinderException("REJECTS_TABLE option is not supported when UNION_BY_NAME is set to true"); - } - } - if (!options.rejects_recovery_columns.empty()) { - if (options.rejects_table_name.empty()) { - throw BinderException( - "REJECTS_RECOVERY_COLUMNS option is only supported when REJECTS_TABLE is set to a table name"); - } for (auto &recovery_col : options.rejects_recovery_columns) { bool found = false; for (idx_t col_idx = 0; col_idx < return_names.size(); col_idx++) { @@ -77,12 +63,6 @@ void ReadCSVData::FinalizeRead(ClientContext &context) { } } } - - if (options.rejects_limit != 0) { - if (options.rejects_table_name.empty()) { - throw BinderException("REJECTS_LIMIT option is only supported when REJECTS_TABLE is set to a table name"); - } - } } static unique_ptr ReadCSVBind(ClientContext &context, TableFunctionBindInput &input, @@ -93,7 +73,27 @@ static unique_ptr ReadCSVBind(ClientContext &context, TableFunctio result->files = MultiFileReader::GetFileList(context, input.inputs[0], "CSV"); options.FromNamedParameters(input.named_parameters, context, return_types, names); - bool explicitly_set_columns = options.explicitly_set_columns; + + // Validate rejects_table options + if (!options.rejects_table_name.empty()) { + if (!options.ignore_errors) { + throw BinderException("REJECTS_TABLE option is only supported when IGNORE_ERRORS is set to true"); + } + if (options.file_options.union_by_name) { + throw BinderException("REJECTS_TABLE option is not supported when UNION_BY_NAME is set to true"); + } + } + + if (options.rejects_limit != 0) { + if (options.rejects_table_name.empty()) { + throw BinderException("REJECTS_LIMIT option is only supported when REJECTS_TABLE is set to a table name"); + } + } + + if (!options.rejects_recovery_columns.empty() && options.rejects_table_name.empty()) { + throw BinderException( + "REJECTS_RECOVERY_COLUMNS option is only supported when REJECTS_TABLE is set to a table name"); + } options.file_options.AutoDetectHivePartitioning(result->files, context); @@ -107,27 +107,15 @@ static unique_ptr ReadCSVBind(ClientContext &context, TableFunctio // Initialize Buffer Manager and Sniffer auto file_handle = BaseCSVReader::OpenCSV(context, options); result->buffer_manager = make_shared(context, std::move(file_handle), options); - CSVSniffer sniffer(options, result->buffer_manager, result->state_machine_cache, explicitly_set_columns); + CSVSniffer sniffer(options, result->buffer_manager, result->state_machine_cache, {&return_types, &names}); auto sniffer_result = sniffer.SniffCSV(); if (names.empty()) { names = sniffer_result.names; return_types = sniffer_result.return_types; - } else { - if (explicitly_set_columns) { - // The user has influenced the names, can't assume they are valid anymore - if (return_types.size() != names.size()) { - throw BinderException("The amount of names specified (%d) and the observed amount of types (%d) in " - "the file don't match", - names.size(), return_types.size()); - } - } else { - D_ASSERT(return_types.size() == names.size()); - } } - - } else { - D_ASSERT(return_types.size() == names.size()); } + D_ASSERT(return_types.size() == names.size()); + result->csv_types = return_types; result->csv_names = names; @@ -157,12 +145,6 @@ static unique_ptr ReadCSVBind(ClientContext &context, TableFunctio return std::move(result); } -static unique_ptr ReadCSVAutoBind(ClientContext &context, TableFunctionBindInput &input, - vector &return_types, vector &names) { - input.named_parameters["auto_detect"] = Value::BOOLEAN(true); - return ReadCSVBind(context, input, return_types, names); -} - //===--------------------------------------------------------------------===// // Parallel CSV Reader CSV Global State //===--------------------------------------------------------------------===// @@ -418,7 +400,8 @@ bool ParallelCSVGlobalState::Next(ClientContext &context, const ReadCSVData &bin first_position = 0; local_batch_index = 0; - line_info.lines_read[file_index++][local_batch_index] = (bind_data.options.has_header ? 1 : 0); + line_info.lines_read[file_index++][local_batch_index] = + (bind_data.options.has_header && bind_data.options.dialect_options.header ? 1 : 0); current_buffer = buffer_manager->GetBuffer(cur_buffer_idx); next_buffer = buffer_manager->GetBuffer(cur_buffer_idx + 1); @@ -981,7 +964,7 @@ TableFunction ReadCSVTableFunction::GetFunction() { TableFunction ReadCSVTableFunction::GetAutoFunction() { auto read_csv_auto = ReadCSVTableFunction::GetFunction(); read_csv_auto.name = "read_csv_auto"; - read_csv_auto.bind = ReadCSVAutoBind; + read_csv_auto.bind = ReadCSVBind; return read_csv_auto; } diff --git a/src/duckdb/src/function/table/system/pragma_user_agent.cpp b/src/duckdb/src/function/table/system/pragma_user_agent.cpp deleted file mode 100644 index 3803f7195..000000000 --- a/src/duckdb/src/function/table/system/pragma_user_agent.cpp +++ /dev/null @@ -1,50 +0,0 @@ -#include "duckdb/function/table/system_functions.hpp" -#include "duckdb/main/config.hpp" - -namespace duckdb { - -struct PragmaUserAgentData : public GlobalTableFunctionState { - PragmaUserAgentData() : finished(false) { - } - - std::string user_agent; - bool finished; -}; - -static unique_ptr PragmaUserAgentBind(ClientContext &context, TableFunctionBindInput &input, - vector &return_types, vector &names) { - - names.emplace_back("user_agent"); - return_types.emplace_back(LogicalType::VARCHAR); - - return nullptr; -} - -unique_ptr PragmaUserAgentInit(ClientContext &context, TableFunctionInitInput &input) { - auto result = make_uniq(); - auto &config = DBConfig::GetConfig(context); - result->user_agent = config.UserAgent(); - - return std::move(result); -} - -void PragmaUserAgentFunction(ClientContext &context, TableFunctionInput &data_p, DataChunk &output) { - auto &data = data_p.global_state->Cast(); - - if (data.finished) { - // signal end of output - return; - } - - output.SetCardinality(1); - output.SetValue(0, 0, data.user_agent); - - data.finished = true; -} - -void PragmaUserAgent::RegisterFunction(BuiltinFunctions &set) { - set.AddFunction( - TableFunction("pragma_user_agent", {}, PragmaUserAgentFunction, PragmaUserAgentBind, PragmaUserAgentInit)); -} - -} // namespace duckdb diff --git a/src/duckdb/src/function/table/system_functions.cpp b/src/duckdb/src/function/table/system_functions.cpp index bf46bee93..7b6a5b045 100644 --- a/src/duckdb/src/function/table/system_functions.cpp +++ b/src/duckdb/src/function/table/system_functions.cpp @@ -18,7 +18,6 @@ void BuiltinFunctions::RegisterSQLiteFunctions() { PragmaDatabaseSize::RegisterFunction(*this); PragmaLastProfilingOutput::RegisterFunction(*this); PragmaDetailedProfilingOutput::RegisterFunction(*this); - PragmaUserAgent::RegisterFunction(*this); DuckDBColumnsFun::RegisterFunction(*this); DuckDBConstraintsFun::RegisterFunction(*this); diff --git a/src/duckdb/src/function/table/version/pragma_version.cpp b/src/duckdb/src/function/table/version/pragma_version.cpp index b666d1905..6d6d5e63a 100644 --- a/src/duckdb/src/function/table/version/pragma_version.cpp +++ b/src/duckdb/src/function/table/version/pragma_version.cpp @@ -1,8 +1,8 @@ #ifndef DUCKDB_VERSION -#define DUCKDB_VERSION "v0.9.3-dev42" +#define DUCKDB_VERSION "v0.9.2-dev684" #endif #ifndef DUCKDB_SOURCE_ID -#define DUCKDB_SOURCE_ID "958f189880" +#define DUCKDB_SOURCE_ID "be47ec4ae4" #endif #include "duckdb/function/table/system_functions.hpp" #include "duckdb/main/database.hpp" diff --git a/src/duckdb/src/include/duckdb.h b/src/duckdb/src/include/duckdb.h index 86b0b1b4a..48cafdf04 100644 --- a/src/duckdb/src/include/duckdb.h +++ b/src/duckdb/src/include/duckdb.h @@ -953,7 +953,7 @@ Returns NULL if the index is out of range for the provided prepared statement. * prepared_statement: The prepared statement for which to get the parameter name from. */ -DUCKDB_API const char *duckdb_parameter_name(duckdb_prepared_statement prepared_statement, idx_t index); +const char *duckdb_parameter_name(duckdb_prepared_statement prepared_statement, idx_t index); /*! Returns the parameter type for the parameter at the given index. @@ -2351,9 +2351,17 @@ Fetch the internal arrow schema from the prepared statement. */ DUCKDB_API duckdb_state duckdb_prepared_arrow_schema(duckdb_prepared_statement prepared, duckdb_arrow_schema *out_schema); +/*! +Convert a data chunk into an arrow struct array. + +* result: The result object the data chunk have been fetched from. +* chunk: The data chunk to convert. +* out_array: The output array. +*/ +DUCKDB_API void duckdb_result_arrow_array(duckdb_result result, duckdb_data_chunk chunk, duckdb_arrow_array *out_array); /*! -Fetch an internal arrow array from the arrow result. +Fetch an internal arrow struct array from the arrow result. This function can be called multiple time to get next chunks, which will free the previous out_array. So consume the out_array before calling this function again. diff --git a/src/duckdb/src/include/duckdb/common/adbc/adbc.h b/src/duckdb/src/include/duckdb/common/adbc/adbc.h index b5f9faf42..b1c6fa11b 100644 --- a/src/duckdb/src/include/duckdb/common/adbc/adbc.h +++ b/src/duckdb/src/include/duckdb/common/adbc/adbc.h @@ -1,3 +1,9 @@ +//////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////// +// THIS FILE IS GENERATED BY apache/arrow, DO NOT EDIT MANUALLY // +//////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////// + // Licensed to the Apache Software Foundation (ASF) under one // or more contributor license agreements. See the NOTICE file // distributed with this work for additional information @@ -35,7 +41,7 @@ /// but not concurrent access. Specific implementations may permit /// multiple threads. /// -/// \version 1.0.0 +/// \version 1.1.0 #pragma once @@ -57,12 +63,99 @@ namespace duckdb_adbc { extern "C" { #endif -#ifndef ARROW_ADBC -#define ARROW_ADBC +// Extra guard for versions of Arrow without the canonical guard +#ifndef ARROW_FLAG_DICTIONARY_ORDERED + +#ifndef ARROW_C_DATA_INTERFACE +#define ARROW_C_DATA_INTERFACE + +#define ARROW_FLAG_DICTIONARY_ORDERED 1 +#define ARROW_FLAG_NULLABLE 2 +#define ARROW_FLAG_MAP_KEYS_SORTED 4 + +struct ArrowSchema { + // Array type description + const char *format; + const char *name; + const char *metadata; + int64_t flags; + int64_t n_children; + struct ArrowSchema **children; + struct ArrowSchema *dictionary; + + // Release callback + void (*release)(struct ArrowSchema *); + // Opaque producer-specific data + void *private_data; +}; + +struct ArrowArray { + // Array data description + int64_t length; + int64_t null_count; + int64_t offset; + int64_t n_buffers; + int64_t n_children; + const void **buffers; + struct ArrowArray **children; + struct ArrowArray *dictionary; + + // Release callback + void (*release)(struct ArrowArray *); + // Opaque producer-specific data + void *private_data; +}; + +#endif // ARROW_C_DATA_INTERFACE + +#ifndef ARROW_C_STREAM_INTERFACE +#define ARROW_C_STREAM_INTERFACE + +struct ArrowArrayStream { + // Callback to get the stream type + // (will be the same for all arrays in the stream). + // + // Return value: 0 if successful, an `errno`-compatible error code otherwise. + // + // If successful, the ArrowSchema must be released independently from the stream. + int (*get_schema)(struct ArrowArrayStream *, struct ArrowSchema *out); + + // Callback to get the next array + // (if no error and the array is released, the stream has ended) + // + // Return value: 0 if successful, an `errno`-compatible error code otherwise. + // + // If successful, the ArrowArray must be released independently from the stream. + int (*get_next)(struct ArrowArrayStream *, struct ArrowArray *out); + + // Callback to get optional detailed error information. + // This must only be called if the last stream operation failed + // with a non-0 return code. + // + // Return value: pointer to a null-terminated character array describing + // the last error, or NULL if no description is available. + // + // The returned pointer is only valid until the next operation on this stream + // (including release). + const char *(*get_last_error)(struct ArrowArrayStream *); + + // Release callback: release the stream's own resources. + // Note that arrays returned by `get_next` must be individually released. + void (*release)(struct ArrowArrayStream *); + + // Opaque producer-specific data + void *private_data; +}; + +#endif // ARROW_C_STREAM_INTERFACE +#endif // ARROW_FLAG_DICTIONARY_ORDERED + +//! @endcond + +/// @} #ifndef ADBC_EXPORTING #define ADBC_EXPORTING -#endif // Storage class macros for Windows // Allow overriding/aliasing with application-defined macros @@ -163,7 +256,24 @@ typedef uint8_t AdbcStatusCode; /// May indicate a database-side error only. #define ADBC_STATUS_UNAUTHORIZED 14 +/// \brief Inform the driver/driver manager that we are using the extended +/// AdbcError struct from ADBC 1.1.0. +/// +/// See the AdbcError documentation for usage. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA INT32_MIN + /// \brief A detailed error message for an operation. +/// +/// The caller must zero-initialize this struct (clarified in ADBC 1.1.0). +/// +/// The structure was extended in ADBC 1.1.0. Drivers and clients using ADBC +/// 1.0.0 will not have the private_data or private_driver fields. Drivers +/// should read/write these fields if and only if vendor_code is equal to +/// ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA. Clients are required to initialize +/// this struct to avoid the possibility of uninitialized values confusing the +/// driver. struct ADBC_EXPORT AdbcError { /// \brief The error message. char *message; @@ -181,8 +291,105 @@ struct ADBC_EXPORT AdbcError { /// Unlike other structures, this is an embedded callback to make it /// easier for the driver manager and driver to cooperate. void (*release)(struct AdbcError *error); + + /// \brief Opaque implementation-defined state. + /// + /// This field may not be used unless vendor_code is + /// ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA. If present, this field is NULLPTR + /// iff the error is unintialized/freed. + /// + /// \since ADBC API revision 1.1.0 + void *private_data; + + /// \brief The associated driver (used by the driver manager to help + /// track state). + /// + /// This field may not be used unless vendor_code is + /// ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA. + /// + /// \since ADBC API revision 1.1.0 + struct AdbcDriver *private_driver; }; +#ifdef __cplusplus +/// \brief A helper to initialize the full AdbcError structure. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_ERROR_INIT \ + (AdbcError {nullptr, ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA, {0, 0, 0, 0, 0}, nullptr, nullptr, nullptr}) +#else +/// \brief A helper to initialize the full AdbcError structure. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_ERROR_INIT \ + ((struct AdbcError) {NULL, ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA, {0, 0, 0, 0, 0}, NULL, NULL, NULL}) +#endif + +/// \brief The size of the AdbcError structure in ADBC 1.0.0. +/// +/// Drivers written for ADBC 1.1.0 and later should never touch more than this +/// portion of an AdbcDriver struct when vendor_code is not +/// ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_ERROR_1_0_0_SIZE (offsetof(struct AdbcError, private_data)) +/// \brief The size of the AdbcError structure in ADBC 1.1.0. +/// +/// Drivers written for ADBC 1.1.0 and later should never touch more than this +/// portion of an AdbcDriver struct when vendor_code is +/// ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_ERROR_1_1_0_SIZE (sizeof(struct AdbcError)) + +/// \brief Extra key-value metadata for an error. +/// +/// The fields here are owned by the driver and should not be freed. The +/// fields here are invalidated when the release callback in AdbcError is +/// called. +/// +/// \since ADBC API revision 1.1.0 +struct ADBC_EXPORT AdbcErrorDetail { + /// \brief The metadata key. + const char *key; + /// \brief The binary metadata value. + const uint8_t *value; + /// \brief The length of the metadata value. + size_t value_length; +}; + +/// \brief Get the number of metadata values available in an error. +/// +/// \since ADBC API revision 1.1.0 +ADBC_EXPORT +int AdbcErrorGetDetailCount(const struct AdbcError *error); + +/// \brief Get a metadata value in an error by index. +/// +/// If index is invalid, returns an AdbcErrorDetail initialized with NULL/0 +/// fields. +/// +/// \since ADBC API revision 1.1.0 +ADBC_EXPORT +struct AdbcErrorDetail AdbcErrorGetDetail(const struct AdbcError *error, int index); + +/// \brief Get an ADBC error from an ArrowArrayStream created by a driver. +/// +/// This allows retrieving error details and other metadata that would +/// normally be suppressed by the Arrow C Stream Interface. +/// +/// The caller MUST NOT release the error; it is managed by the release +/// callback in the stream itself. +/// +/// \param[in] stream The stream to query. +/// \param[out] status The ADBC status code, or ADBC_STATUS_OK if there is no +/// error. Not written to if the stream does not contain an ADBC error or +/// if the pointer is NULL. +/// \return NULL if not supported. +/// \since ADBC API revision 1.1.0 +ADBC_EXPORT +const struct AdbcError *AdbcErrorFromArrayStream(struct ArrowArrayStream *stream, AdbcStatusCode *status); + /// @} /// \defgroup adbc-constants Constants @@ -194,6 +401,14 @@ struct ADBC_EXPORT AdbcError { /// point to an AdbcDriver. #define ADBC_VERSION_1_0_0 1000000 +/// \brief ADBC revision 1.1.0. +/// +/// When passed to an AdbcDriverInitFunc(), the driver parameter must +/// point to an AdbcDriver. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_VERSION_1_1_0 1001000 + /// \brief Canonical option value for enabling an option. /// /// For use as the value in SetOption calls. @@ -203,6 +418,34 @@ struct ADBC_EXPORT AdbcError { /// For use as the value in SetOption calls. #define ADBC_OPTION_VALUE_DISABLED "false" +/// \brief Canonical option name for URIs. +/// +/// Should be used as the expected option name to specify a URI for +/// any ADBC driver. +/// +/// The type is char*. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_OPTION_URI "uri" +/// \brief Canonical option name for usernames. +/// +/// Should be used as the expected option name to specify a username +/// to a driver for authentication. +/// +/// The type is char*. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_OPTION_USERNAME "username" +/// \brief Canonical option name for passwords. +/// +/// Should be used as the expected option name to specify a password +/// for authentication to a driver. +/// +/// The type is char*. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_OPTION_PASSWORD "password" + /// \brief The database vendor/product name (e.g. the server name). /// (type: utf8). /// @@ -230,6 +473,15 @@ struct ADBC_EXPORT AdbcError { /// /// \see AdbcConnectionGetInfo #define ADBC_INFO_DRIVER_ARROW_VERSION 102 +/// \brief The driver ADBC API version (type: int64). +/// +/// The value should be one of the ADBC_VERSION constants. +/// +/// \since ADBC API revision 1.1.0 +/// \see AdbcConnectionGetInfo +/// \see ADBC_VERSION_1_0_0 +/// \see ADBC_VERSION_1_1_0 +#define ADBC_INFO_DRIVER_ADBC_VERSION 103 /// \brief Return metadata on catalogs, schemas, tables, and columns. /// @@ -252,18 +504,133 @@ struct ADBC_EXPORT AdbcError { /// \see AdbcConnectionGetObjects #define ADBC_OBJECT_DEPTH_COLUMNS ADBC_OBJECT_DEPTH_ALL +/// \defgroup adbc-table-statistics ADBC Statistic Types +/// Standard statistic names for AdbcConnectionGetStatistics. +/// @{ + +/// \brief The dictionary-encoded name of the average byte width statistic. +#define ADBC_STATISTIC_AVERAGE_BYTE_WIDTH_KEY 0 +/// \brief The average byte width statistic. The average size in bytes of a +/// row in the column. Value type is float64. +/// +/// For example, this is roughly the average length of a string for a string +/// column. +#define ADBC_STATISTIC_AVERAGE_BYTE_WIDTH_NAME "adbc.statistic.byte_width" +/// \brief The dictionary-encoded name of the distinct value count statistic. +#define ADBC_STATISTIC_DISTINCT_COUNT_KEY 1 +/// \brief The distinct value count (NDV) statistic. The number of distinct +/// values in the column. Value type is int64 (when not approximate) or +/// float64 (when approximate). +#define ADBC_STATISTIC_DISTINCT_COUNT_NAME "adbc.statistic.distinct_count" +/// \brief The dictionary-encoded name of the max byte width statistic. +#define ADBC_STATISTIC_MAX_BYTE_WIDTH_KEY 2 +/// \brief The max byte width statistic. The maximum size in bytes of a row +/// in the column. Value type is int64 (when not approximate) or float64 +/// (when approximate). +/// +/// For example, this is the maximum length of a string for a string column. +#define ADBC_STATISTIC_MAX_BYTE_WIDTH_NAME "adbc.statistic.byte_width" +/// \brief The dictionary-encoded name of the max value statistic. +#define ADBC_STATISTIC_MAX_VALUE_KEY 3 +/// \brief The max value statistic. Value type is column-dependent. +#define ADBC_STATISTIC_MAX_VALUE_NAME "adbc.statistic.byte_width" +/// \brief The dictionary-encoded name of the min value statistic. +#define ADBC_STATISTIC_MIN_VALUE_KEY 4 +/// \brief The min value statistic. Value type is column-dependent. +#define ADBC_STATISTIC_MIN_VALUE_NAME "adbc.statistic.byte_width" +/// \brief The dictionary-encoded name of the null count statistic. +#define ADBC_STATISTIC_NULL_COUNT_KEY 5 +/// \brief The null count statistic. The number of values that are null in +/// the column. Value type is int64 (when not approximate) or float64 +/// (when approximate). +#define ADBC_STATISTIC_NULL_COUNT_NAME "adbc.statistic.null_count" +/// \brief The dictionary-encoded name of the row count statistic. +#define ADBC_STATISTIC_ROW_COUNT_KEY 6 +/// \brief The row count statistic. The number of rows in the column or +/// table. Value type is int64 (when not approximate) or float64 (when +/// approximate). +#define ADBC_STATISTIC_ROW_COUNT_NAME "adbc.statistic.row_count" +/// @} + /// \brief The name of the canonical option for whether autocommit is /// enabled. /// +/// The type is char*. +/// /// \see AdbcConnectionSetOption #define ADBC_CONNECTION_OPTION_AUTOCOMMIT "adbc.connection.autocommit" /// \brief The name of the canonical option for whether the current /// connection should be restricted to being read-only. /// +/// The type is char*. +/// /// \see AdbcConnectionSetOption #define ADBC_CONNECTION_OPTION_READ_ONLY "adbc.connection.readonly" +/// \brief The name of the canonical option for the current catalog. +/// +/// The type is char*. +/// +/// \see AdbcConnectionGetOption +/// \see AdbcConnectionSetOption +/// \since ADBC API revision 1.1.0 +#define ADBC_CONNECTION_OPTION_CURRENT_CATALOG "adbc.connection.catalog" + +/// \brief The name of the canonical option for the current schema. +/// +/// The type is char*. +/// +/// \see AdbcConnectionGetOption +/// \see AdbcConnectionSetOption +/// \since ADBC API revision 1.1.0 +#define ADBC_CONNECTION_OPTION_CURRENT_DB_SCHEMA "adbc.connection.db_schema" + +/// \brief The name of the canonical option for making query execution +/// nonblocking. +/// +/// When enabled, AdbcStatementExecutePartitions will return +/// partitions as soon as they are available, instead of returning +/// them all at the end. When there are no more to return, it will +/// return an empty set of partitions. AdbcStatementExecuteQuery and +/// AdbcStatementExecuteSchema are not affected. +/// +/// The default is ADBC_OPTION_VALUE_DISABLED. +/// +/// The type is char*. +/// +/// \see AdbcStatementSetOption +/// \since ADBC API revision 1.1.0 +#define ADBC_STATEMENT_OPTION_INCREMENTAL "adbc.statement.exec.incremental" + +/// \brief The name of the option for getting the progress of a query. +/// +/// The value is not necessarily in any particular range or have any +/// particular units. (For example, it might be a percentage, bytes of data, +/// rows of data, number of workers, etc.) The max value can be retrieved via +/// ADBC_STATEMENT_OPTION_MAX_PROGRESS. This represents the progress of +/// execution, not of consumption (i.e., it is independent of how much of the +/// result set has been read by the client via ArrowArrayStream.get_next().) +/// +/// The type is double. +/// +/// \see AdbcStatementGetOptionDouble +/// \since ADBC API revision 1.1.0 +#define ADBC_STATEMENT_OPTION_PROGRESS "adbc.statement.exec.progress" + +/// \brief The name of the option for getting the maximum progress of a query. +/// +/// This is the value of ADBC_STATEMENT_OPTION_PROGRESS for a completed query. +/// If not supported, or if the value is nonpositive, then the maximum is not +/// known. (For instance, the query may be fully streaming and the driver +/// does not know when the result set will end.) +/// +/// The type is double. +/// +/// \see AdbcStatementGetOptionDouble +/// \since ADBC API revision 1.1.0 +#define ADBC_STATEMENT_OPTION_MAX_PROGRESS "adbc.statement.exec.max_progress" + /// \brief The name of the canonical option for setting the isolation /// level of a transaction. /// @@ -272,6 +639,8 @@ struct ADBC_EXPORT AdbcError { /// isolation level is not supported by a driver, it should return an /// appropriate error. /// +/// The type is char*. +/// /// \see AdbcConnectionSetOption #define ADBC_CONNECTION_OPTION_ISOLATION_LEVEL "adbc.connection.transaction.isolation_level" @@ -356,8 +725,12 @@ struct ADBC_EXPORT AdbcError { /// exist. If the table exists but has a different schema, /// ADBC_STATUS_ALREADY_EXISTS should be raised. Else, data should be /// appended to the target table. +/// +/// The type is char*. #define ADBC_INGEST_OPTION_TARGET_TABLE "adbc.ingest.target_table" /// \brief Whether to create (the default) or append. +/// +/// The type is char*. #define ADBC_INGEST_OPTION_MODE "adbc.ingest.mode" /// \brief Create the table and insert data; error if the table exists. #define ADBC_INGEST_OPTION_MODE_CREATE "adbc.ingest.mode.create" @@ -365,6 +738,15 @@ struct ADBC_EXPORT AdbcError { /// table does not exist (ADBC_STATUS_NOT_FOUND) or does not match /// the schema of the data to append (ADBC_STATUS_ALREADY_EXISTS). #define ADBC_INGEST_OPTION_MODE_APPEND "adbc.ingest.mode.append" +/// \brief Create the table and insert data; drop the original table +/// if it already exists. +/// \since ADBC API revision 1.1.0 +#define ADBC_INGEST_OPTION_MODE_REPLACE "adbc.ingest.mode.replace" +/// \brief Insert data; create the table if it does not exist, or +/// error if the table exists, but the schema does not match the +/// schema of the data to append (ADBC_STATUS_ALREADY_EXISTS). +/// \since ADBC API revision 1.1.0 +#define ADBC_INGEST_OPTION_MODE_CREATE_APPEND "adbc.ingest.mode.create_append" /// @} @@ -530,7 +912,7 @@ struct ADBC_EXPORT AdbcDriver { AdbcStatusCode (*DatabaseRelease)(struct AdbcDatabase *, struct AdbcError *); AdbcStatusCode (*ConnectionCommit)(struct AdbcConnection *, struct AdbcError *); - AdbcStatusCode (*ConnectionGetInfo)(struct AdbcConnection *, uint32_t *, size_t, struct ArrowArrayStream *, + AdbcStatusCode (*ConnectionGetInfo)(struct AdbcConnection *, const uint32_t *, size_t, struct ArrowArrayStream *, struct AdbcError *); AdbcStatusCode (*ConnectionGetObjects)(struct AdbcConnection *, int, const char *, const char *, const char *, const char **, const char *, struct ArrowArrayStream *, struct AdbcError *); @@ -559,8 +941,89 @@ struct ADBC_EXPORT AdbcDriver { AdbcStatusCode (*StatementSetOption)(struct AdbcStatement *, const char *, const char *, struct AdbcError *); AdbcStatusCode (*StatementSetSqlQuery)(struct AdbcStatement *, const char *, struct AdbcError *); AdbcStatusCode (*StatementSetSubstraitPlan)(struct AdbcStatement *, const uint8_t *, size_t, struct AdbcError *); + + /// \defgroup adbc-1.1.0 ADBC API Revision 1.1.0 + /// + /// Functions added in ADBC 1.1.0. For backwards compatibility, + /// these members must not be accessed unless the version passed to + /// the AdbcDriverInitFunc is greater than or equal to + /// ADBC_VERSION_1_1_0. + /// + /// For a 1.0.0 driver being loaded by a 1.1.0 driver manager: the + /// 1.1.0 manager will allocate the new, expanded AdbcDriver struct + /// and attempt to have the driver initialize it with + /// ADBC_VERSION_1_1_0. This must return an error, after which the + /// driver will try again with ADBC_VERSION_1_0_0. The driver must + /// not access the new fields, which will carry undefined values. + /// + /// For a 1.1.0 driver being loaded by a 1.0.0 driver manager: the + /// 1.0.0 manager will allocate the old AdbcDriver struct and + /// attempt to have the driver initialize it with + /// ADBC_VERSION_1_0_0. The driver must not access the new fields, + /// and should initialize the old fields. + /// + /// @{ + + int (*ErrorGetDetailCount)(const struct AdbcError *error); + struct AdbcErrorDetail (*ErrorGetDetail)(const struct AdbcError *error, int index); + const struct AdbcError *(*ErrorFromArrayStream)(struct ArrowArrayStream *stream, AdbcStatusCode *status); + + AdbcStatusCode (*DatabaseGetOption)(struct AdbcDatabase *, const char *, char *, size_t *, struct AdbcError *); + AdbcStatusCode (*DatabaseGetOptionBytes)(struct AdbcDatabase *, const char *, uint8_t *, size_t *, + struct AdbcError *); + AdbcStatusCode (*DatabaseGetOptionDouble)(struct AdbcDatabase *, const char *, double *, struct AdbcError *); + AdbcStatusCode (*DatabaseGetOptionInt)(struct AdbcDatabase *, const char *, int64_t *, struct AdbcError *); + AdbcStatusCode (*DatabaseSetOptionBytes)(struct AdbcDatabase *, const char *, const uint8_t *, size_t, + struct AdbcError *); + AdbcStatusCode (*DatabaseSetOptionDouble)(struct AdbcDatabase *, const char *, double, struct AdbcError *); + AdbcStatusCode (*DatabaseSetOptionInt)(struct AdbcDatabase *, const char *, int64_t, struct AdbcError *); + + AdbcStatusCode (*ConnectionCancel)(struct AdbcConnection *, struct AdbcError *); + AdbcStatusCode (*ConnectionGetOption)(struct AdbcConnection *, const char *, char *, size_t *, struct AdbcError *); + AdbcStatusCode (*ConnectionGetOptionBytes)(struct AdbcConnection *, const char *, uint8_t *, size_t *, + struct AdbcError *); + AdbcStatusCode (*ConnectionGetOptionDouble)(struct AdbcConnection *, const char *, double *, struct AdbcError *); + AdbcStatusCode (*ConnectionGetOptionInt)(struct AdbcConnection *, const char *, int64_t *, struct AdbcError *); + AdbcStatusCode (*ConnectionGetStatistics)(struct AdbcConnection *, const char *, const char *, const char *, char, + struct ArrowArrayStream *, struct AdbcError *); + AdbcStatusCode (*ConnectionGetStatisticNames)(struct AdbcConnection *, struct ArrowArrayStream *, + struct AdbcError *); + AdbcStatusCode (*ConnectionSetOptionBytes)(struct AdbcConnection *, const char *, const uint8_t *, size_t, + struct AdbcError *); + AdbcStatusCode (*ConnectionSetOptionDouble)(struct AdbcConnection *, const char *, double, struct AdbcError *); + AdbcStatusCode (*ConnectionSetOptionInt)(struct AdbcConnection *, const char *, int64_t, struct AdbcError *); + + AdbcStatusCode (*StatementCancel)(struct AdbcStatement *, struct AdbcError *); + AdbcStatusCode (*StatementExecuteSchema)(struct AdbcStatement *, struct ArrowSchema *, struct AdbcError *); + AdbcStatusCode (*StatementGetOption)(struct AdbcStatement *, const char *, char *, size_t *, struct AdbcError *); + AdbcStatusCode (*StatementGetOptionBytes)(struct AdbcStatement *, const char *, uint8_t *, size_t *, + struct AdbcError *); + AdbcStatusCode (*StatementGetOptionDouble)(struct AdbcStatement *, const char *, double *, struct AdbcError *); + AdbcStatusCode (*StatementGetOptionInt)(struct AdbcStatement *, const char *, int64_t *, struct AdbcError *); + AdbcStatusCode (*StatementSetOptionBytes)(struct AdbcStatement *, const char *, const uint8_t *, size_t, + struct AdbcError *); + AdbcStatusCode (*StatementSetOptionDouble)(struct AdbcStatement *, const char *, double, struct AdbcError *); + AdbcStatusCode (*StatementSetOptionInt)(struct AdbcStatement *, const char *, int64_t, struct AdbcError *); + + /// @} }; +/// \brief The size of the AdbcDriver structure in ADBC 1.0.0. +/// Drivers written for ADBC 1.1.0 and later should never touch more +/// than this portion of an AdbcDriver struct when given +/// ADBC_VERSION_1_0_0. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_DRIVER_1_0_0_SIZE (offsetof(struct AdbcDriver, ErrorGetDetailCount)) + +/// \brief The size of the AdbcDriver structure in ADBC 1.1.0. +/// Drivers written for ADBC 1.1.0 and later should never touch more +/// than this portion of an AdbcDriver struct when given +/// ADBC_VERSION_1_1_0. +/// +/// \since ADBC API revision 1.1.0 +#define ADBC_DRIVER_1_1_0_SIZE (sizeof(struct AdbcDriver)) + /// @} /// \addtogroup adbc-database @@ -576,16 +1039,186 @@ struct ADBC_EXPORT AdbcDriver { ADBC_EXPORT AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase *database, struct AdbcError *error); +/// \brief Get a string option of the database. +/// +/// This must always be thread-safe (other operations are not), though +/// given the semantics here, it is not recommended to call GetOption +/// concurrently with itself. +/// +/// length must be provided and must be the size of the buffer pointed +/// to by value. If there is sufficient space, the driver will copy +/// the option value (including the null terminator) to buffer and set +/// length to the size of the actual value. If the buffer is too +/// small, no data will be written and length will be set to the +/// required length. +/// +/// In other words: +/// +/// - If output length <= input length, value will contain a value +/// with length bytes. +/// - If output length > input length, nothing has been written to +/// value. +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] database The database. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[in,out] length The length of value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseGetOption(struct AdbcDatabase *database, const char *key, char *value, size_t *length, + struct AdbcError *error); + +/// \brief Get a bytestring option of the database. +/// +/// This must always be thread-safe (other operations are not), though +/// given the semantics here, it is not recommended to call +/// GetOptionBytes concurrently with itself. +/// +/// length must be provided and must be the size of the buffer pointed +/// to by value. If there is sufficient space, the driver will copy +/// the option value to buffer and set length to the size of the +/// actual value. If the buffer is too small, no data will be written +/// and length will be set to the required length. +/// +/// In other words: +/// +/// - If output length <= input length, value will contain a value +/// with length bytes. +/// - If output length > input length, nothing has been written to +/// value. +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] database The database. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[in,out] length The option value length. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseGetOptionBytes(struct AdbcDatabase *database, const char *key, uint8_t *value, + size_t *length, struct AdbcError *error); + +/// \brief Get a double option of the database. +/// +/// This must always be thread-safe (other operations are not). +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the double +/// representation of an integer option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] database The database. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseGetOptionDouble(struct AdbcDatabase *database, const char *key, double *value, + struct AdbcError *error); + +/// \brief Get an integer option of the database. +/// +/// This must always be thread-safe (other operations are not). +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the integer +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] database The database. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseGetOptionInt(struct AdbcDatabase *database, const char *key, int64_t *value, + struct AdbcError *error); + /// \brief Set a char* option. /// /// Options may be set before AdbcDatabaseInit. Some drivers may /// support setting options after initialization as well. /// +/// \param[in] database The database. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. /// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized ADBC_EXPORT AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase *database, const char *key, const char *value, struct AdbcError *error); +/// \brief Set a bytestring option on a database. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] database The database. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[in] length The option value length. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseSetOptionBytes(struct AdbcDatabase *database, const char *key, const uint8_t *value, + size_t length, struct AdbcError *error); + +/// \brief Set a double option on a database. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] database The database. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseSetOptionDouble(struct AdbcDatabase *database, const char *key, double value, + struct AdbcError *error); + +/// \brief Set an integer option on a database. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] database The database. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseSetOptionInt(struct AdbcDatabase *database, const char *key, int64_t value, + struct AdbcError *error); + /// \brief Finish setting options and initialize the database. /// /// Some drivers may support setting options after initialization @@ -620,11 +1253,62 @@ AdbcStatusCode AdbcConnectionNew(struct AdbcConnection *connection, struct AdbcE /// Options may be set before AdbcConnectionInit. Some drivers may /// support setting options after initialization as well. /// +/// \param[in] connection The database connection. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. /// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized ADBC_EXPORT AdbcStatusCode AdbcConnectionSetOption(struct AdbcConnection *connection, const char *key, const char *value, struct AdbcError *error); +/// \brief Set a bytestring option on a connection. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The connection. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[in] length The option value length. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcConnectionSetOptionBytes(struct AdbcConnection *connection, const char *key, const uint8_t *value, + size_t length, struct AdbcError *error); + +/// \brief Set an integer option. +/// +/// Options may be set before AdbcConnectionInit. Some drivers may +/// support setting options after initialization as well. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The database connection. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcConnectionSetOptionInt(struct AdbcConnection *connection, const char *key, int64_t value, + struct AdbcError *error); + +/// \brief Set a double option. +/// +/// Options may be set before AdbcConnectionInit. Some drivers may +/// support setting options after initialization as well. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The database connection. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcConnectionSetOptionDouble(struct AdbcConnection *connection, const char *key, double value, + struct AdbcError *error); + /// \brief Finish setting options and initialize the connection. /// /// Some drivers may support setting options after initialization @@ -641,6 +1325,29 @@ AdbcStatusCode AdbcConnectionInit(struct AdbcConnection *connection, struct Adbc ADBC_EXPORT AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection *connection, struct AdbcError *error); +/// \brief Cancel the in-progress operation on a connection. +/// +/// This can be called during AdbcConnectionGetObjects (or similar), +/// or while consuming an ArrowArrayStream returned from such. +/// Calling this function should make the other functions return +/// ADBC_STATUS_CANCELLED (from ADBC functions) or ECANCELED (from +/// methods of ArrowArrayStream). (It is not guaranteed to, for +/// instance, the result set may be buffered in memory already.) +/// +/// This must always be thread-safe (other operations are not). It is +/// not necessarily signal-safe. +/// +/// \since ADBC API revision 1.1.0 +/// +/// \param[in] connection The connection to cancel. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// +/// \return ADBC_STATUS_INVALID_STATE if there is no operation to cancel. +/// \return ADBC_STATUS_UNKNOWN if the operation could not be cancelled. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionCancel(struct AdbcConnection *connection, struct AdbcError *error); + /// \defgroup adbc-connection-metadata Metadata /// Functions for retrieving metadata about the database. /// @@ -654,6 +1361,8 @@ AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection *connection, struct A /// concurrent active statements and it must execute a SQL query /// internally in order to implement the metadata function). /// +/// This AdbcConnection must outlive the returned ArrowArrayStream. +/// /// Some functions accept "search pattern" arguments, which are /// strings that can contain the special character "%" to match zero /// or more characters, or "_" to match exactly one character. (See @@ -688,6 +1397,10 @@ AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection *connection, struct A /// for ADBC usage. Drivers/vendors will ignore requests for /// unrecognized codes (the row will be omitted from the result). /// +/// Since ADBC 1.1.0: the range [500, 1_000) is reserved for "XDBC" +/// information, which is the same metadata provided by the same info +/// code range in the Arrow Flight SQL GetSqlInfo RPC. +/// /// \param[in] connection The connection to query. /// \param[in] info_codes A list of metadata codes to fetch, or NULL /// to fetch all. @@ -696,8 +1409,8 @@ AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection *connection, struct A /// \param[out] out The result set. /// \param[out] error Error details, if an error occurs. ADBC_EXPORT -AdbcStatusCode AdbcConnectionGetInfo(struct AdbcConnection *connection, uint32_t *info_codes, size_t info_codes_length, - struct ArrowArrayStream *out, struct AdbcError *error); +AdbcStatusCode AdbcConnectionGetInfo(struct AdbcConnection *connection, const uint32_t *info_codes, + size_t info_codes_length, struct ArrowArrayStream *out, struct AdbcError *error); /// \brief Get a hierarchical view of all catalogs, database schemas, /// tables, and columns. @@ -778,6 +1491,8 @@ AdbcStatusCode AdbcConnectionGetInfo(struct AdbcConnection *connection, uint32_t /// | fk_table | utf8 not null | /// | fk_column_name | utf8 not null | /// +/// This AdbcConnection must outlive the returned ArrowArrayStream. +/// /// \param[in] connection The database connection. /// \param[in] depth The level of nesting to display. If 0, display /// all levels. If 1, display only catalogs (i.e. catalog_schemas @@ -806,6 +1521,205 @@ AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection *connection, int d const char *db_schema, const char *table_name, const char **table_type, const char *column_name, struct ArrowArrayStream *out, struct AdbcError *error); +/// \brief Get a string option of the connection. +/// +/// This must always be thread-safe (other operations are not), though +/// given the semantics here, it is not recommended to call GetOption +/// concurrently with itself. +/// +/// length must be provided and must be the size of the buffer pointed +/// to by value. If there is sufficient space, the driver will copy +/// the option value (including the null terminator) to buffer and set +/// length to the size of the actual value. If the buffer is too +/// small, no data will be written and length will be set to the +/// required length. +/// +/// In other words: +/// +/// - If output length <= input length, value will contain a value +/// with length bytes. +/// - If output length > input length, nothing has been written to +/// value. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The database connection. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[in,out] length The length of value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection *connection, const char *key, char *value, size_t *length, + struct AdbcError *error); + +/// \brief Get a bytestring option of the connection. +/// +/// This must always be thread-safe (other operations are not), though +/// given the semantics here, it is not recommended to call +/// GetOptionBytes concurrently with itself. +/// +/// length must be provided and must be the size of the buffer pointed +/// to by value. If there is sufficient space, the driver will copy +/// the option value to buffer and set length to the size of the +/// actual value. If the buffer is too small, no data will be written +/// and length will be set to the required length. +/// +/// In other words: +/// +/// - If output length <= input length, value will contain a value +/// with length bytes. +/// - If output length > input length, nothing has been written to +/// value. +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The connection. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[in,out] length The option value length. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetOptionBytes(struct AdbcConnection *connection, const char *key, uint8_t *value, + size_t *length, struct AdbcError *error); + +/// \brief Get an integer option of the connection. +/// +/// This must always be thread-safe (other operations are not). +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The database connection. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetOptionInt(struct AdbcConnection *connection, const char *key, int64_t *value, + struct AdbcError *error); + +/// \brief Get a double option of the connection. +/// +/// This must always be thread-safe (other operations are not). +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The database connection. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetOptionDouble(struct AdbcConnection *connection, const char *key, double *value, + struct AdbcError *error); + +/// \brief Get statistics about the data distribution of table(s). +/// +/// The result is an Arrow dataset with the following schema: +/// +/// | Field Name | Field Type | +/// |--------------------------|----------------------------------| +/// | catalog_name | utf8 | +/// | catalog_db_schemas | list not null | +/// +/// DB_SCHEMA_SCHEMA is a Struct with fields: +/// +/// | Field Name | Field Type | +/// |--------------------------|----------------------------------| +/// | db_schema_name | utf8 | +/// | db_schema_statistics | list not null | +/// +/// STATISTICS_SCHEMA is a Struct with fields: +/// +/// | Field Name | Field Type | Comments | +/// |--------------------------|----------------------------------| -------- | +/// | table_name | utf8 not null | | +/// | column_name | utf8 | (1) | +/// | statistic_key | int16 not null | (2) | +/// | statistic_value | VALUE_SCHEMA not null | | +/// | statistic_is_approximate | bool not null | (3) | +/// +/// 1. If null, then the statistic applies to the entire table. +/// 2. A dictionary-encoded statistic name (although we do not use the Arrow +/// dictionary type). Values in [0, 1024) are reserved for ADBC. Other +/// values are for implementation-specific statistics. For the definitions +/// of predefined statistic types, see \ref adbc-table-statistics. To get +/// driver-specific statistic names, use AdbcConnectionGetStatisticNames. +/// 3. If true, then the value is approximate or best-effort. +/// +/// VALUE_SCHEMA is a dense union with members: +/// +/// | Field Name | Field Type | +/// |--------------------------|----------------------------------| +/// | int64 | int64 | +/// | uint64 | uint64 | +/// | float64 | float64 | +/// | binary | binary | +/// +/// This AdbcConnection must outlive the returned ArrowArrayStream. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The database connection. +/// \param[in] catalog The catalog (or nullptr). May be a search +/// pattern (see section documentation). +/// \param[in] db_schema The database schema (or nullptr). May be a +/// search pattern (see section documentation). +/// \param[in] table_name The table name (or nullptr). May be a +/// search pattern (see section documentation). +/// \param[in] approximate If zero, request exact values of +/// statistics, else allow for best-effort, approximate, or cached +/// values. The database may return approximate values regardless, +/// as indicated in the result. Requesting exact values may be +/// expensive or unsupported. +/// \param[out] out The result set. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetStatistics(struct AdbcConnection *connection, const char *catalog, + const char *db_schema, const char *table_name, char approximate, + struct ArrowArrayStream *out, struct AdbcError *error); + +/// \brief Get the names of statistics specific to this driver. +/// +/// The result is an Arrow dataset with the following schema: +/// +/// Field Name | Field Type +/// ---------------|---------------- +/// statistic_name | utf8 not null +/// statistic_key | int16 not null +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] connection The database connection. +/// \param[out] out The result set. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetStatisticNames(struct AdbcConnection *connection, struct ArrowArrayStream *out, + struct AdbcError *error); + /// \brief Get the Arrow schema of a table. /// /// \param[in] connection The database connection. @@ -827,6 +1741,8 @@ AdbcStatusCode AdbcConnectionGetTableSchema(struct AdbcConnection *connection, c /// ---------------|-------------- /// table_type | utf8 not null /// +/// This AdbcConnection must outlive the returned ArrowArrayStream. +/// /// \param[in] connection The database connection. /// \param[out] out The result set. /// \param[out] error Error details, if an error occurs. @@ -854,6 +1770,8 @@ AdbcStatusCode AdbcConnectionGetTableTypes(struct AdbcConnection *connection, st /// /// A partition can be retrieved from AdbcPartitions. /// +/// This AdbcConnection must outlive the returned ArrowArrayStream. +/// /// \param[in] connection The connection to use. This does not have /// to be the same connection that the partition was created on. /// \param[in] serialized_partition The partition descriptor. @@ -918,7 +1836,11 @@ AdbcStatusCode AdbcStatementRelease(struct AdbcStatement *statement, struct Adbc /// \brief Execute a statement and get the results. /// -/// This invalidates any prior result sets. +/// This invalidates any prior result sets. This AdbcStatement must +/// outlive the returned ArrowArrayStream. +/// +/// Since ADBC 1.1.0: releasing the returned ArrowArrayStream without +/// consuming it fully is equivalent to calling AdbcStatementCancel. /// /// \param[in] statement The statement to execute. /// \param[out] out The results. Pass NULL if the client does not @@ -931,6 +1853,26 @@ ADBC_EXPORT AdbcStatusCode AdbcStatementExecuteQuery(struct AdbcStatement *statement, struct ArrowArrayStream *out, int64_t *rows_affected, struct AdbcError *error); +/// \brief Get the schema of the result set of a query without +/// executing it. +/// +/// This invalidates any prior result sets. +/// +/// Depending on the driver, this may require first executing +/// AdbcStatementPrepare. +/// +/// \since ADBC API revision 1.1.0 +/// +/// \param[in] statement The statement to execute. +/// \param[out] out The result schema. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the driver does not support this. +ADBC_EXPORT +AdbcStatusCode AdbcStatementExecuteSchema(struct AdbcStatement *statement, struct ArrowSchema *schema, + struct AdbcError *error); + /// \brief Turn this statement into a prepared statement to be /// executed multiple times. /// @@ -1008,6 +1950,154 @@ ADBC_EXPORT AdbcStatusCode AdbcStatementBindStream(struct AdbcStatement *statement, struct ArrowArrayStream *stream, struct AdbcError *error); +/// \brief Cancel execution of an in-progress query. +/// +/// This can be called during AdbcStatementExecuteQuery (or similar), +/// or while consuming an ArrowArrayStream returned from such. +/// Calling this function should make the other functions return +/// ADBC_STATUS_CANCELLED (from ADBC functions) or ECANCELED (from +/// methods of ArrowArrayStream). (It is not guaranteed to, for +/// instance, the result set may be buffered in memory already.) +/// +/// This must always be thread-safe (other operations are not). It is +/// not necessarily signal-safe. +/// +/// \since ADBC API revision 1.1.0 +/// +/// \param[in] statement The statement to cancel. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// +/// \return ADBC_STATUS_INVALID_STATE if there is no query to cancel. +/// \return ADBC_STATUS_UNKNOWN if the query could not be cancelled. +ADBC_EXPORT +AdbcStatusCode AdbcStatementCancel(struct AdbcStatement *statement, struct AdbcError *error); + +/// \brief Get a string option of the statement. +/// +/// This must always be thread-safe (other operations are not), though +/// given the semantics here, it is not recommended to call GetOption +/// concurrently with itself. +/// +/// length must be provided and must be the size of the buffer pointed +/// to by value. If there is sufficient space, the driver will copy +/// the option value (including the null terminator) to buffer and set +/// length to the size of the actual value. If the buffer is too +/// small, no data will be written and length will be set to the +/// required length. +/// +/// In other words: +/// +/// - If output length <= input length, value will contain a value +/// with length bytes. +/// - If output length > input length, nothing has been written to +/// value. +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] statement The statement. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[in,out] length The length of value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcStatementGetOption(struct AdbcStatement *statement, const char *key, char *value, size_t *length, + struct AdbcError *error); + +/// \brief Get a bytestring option of the statement. +/// +/// This must always be thread-safe (other operations are not), though +/// given the semantics here, it is not recommended to call +/// GetOptionBytes concurrently with itself. +/// +/// length must be provided and must be the size of the buffer pointed +/// to by value. If there is sufficient space, the driver will copy +/// the option value to buffer and set length to the size of the +/// actual value. If the buffer is too small, no data will be written +/// and length will be set to the required length. +/// +/// In other words: +/// +/// - If output length <= input length, value will contain a value +/// with length bytes. +/// - If output length > input length, nothing has been written to +/// value. +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] statement The statement. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[in,out] length The option value length. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcStatementGetOptionBytes(struct AdbcStatement *statement, const char *key, uint8_t *value, + size_t *length, struct AdbcError *error); + +/// \brief Get an integer option of the statement. +/// +/// This must always be thread-safe (other operations are not). +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] statement The statement. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcStatementGetOptionInt(struct AdbcStatement *statement, const char *key, int64_t *value, + struct AdbcError *error); + +/// \brief Get a double option of the statement. +/// +/// This must always be thread-safe (other operations are not). +/// +/// For standard options, drivers must always support getting the +/// option value (if they support getting option values at all) via +/// the type specified in the option. (For example, an option set via +/// SetOptionDouble must be retrievable via GetOptionDouble.) Drivers +/// may also support getting a converted option value via other +/// getters if needed. (For example, getting the string +/// representation of a double option.) +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] statement The statement. +/// \param[in] key The option to get. +/// \param[out] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized. +ADBC_EXPORT +AdbcStatusCode AdbcStatementGetOptionDouble(struct AdbcStatement *statement, const char *key, double *value, + struct AdbcError *error); + /// \brief Get the schema for bound parameters. /// /// This retrieves an Arrow schema describing the number, names, and @@ -1028,10 +2118,56 @@ AdbcStatusCode AdbcStatementGetParameterSchema(struct AdbcStatement *statement, struct AdbcError *error); /// \brief Set a string option on a statement. +/// \param[in] statement The statement. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized. ADBC_EXPORT AdbcStatusCode AdbcStatementSetOption(struct AdbcStatement *statement, const char *key, const char *value, struct AdbcError *error); +/// \brief Set a bytestring option on a statement. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] statement The statement. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[in] length The option value length. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcStatementSetOptionBytes(struct AdbcStatement *statement, const char *key, const uint8_t *value, + size_t length, struct AdbcError *error); + +/// \brief Set an integer option on a statement. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] statement The statement. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcStatementSetOptionInt(struct AdbcStatement *statement, const char *key, int64_t value, + struct AdbcError *error); + +/// \brief Set a double option on a statement. +/// +/// \since ADBC API revision 1.1.0 +/// \param[in] statement The statement. +/// \param[in] key The option to set. +/// \param[in] value The option value. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcStatementSetOptionDouble(struct AdbcStatement *statement, const char *key, double value, + struct AdbcError *error); + /// \addtogroup adbc-statement-partition /// @{ @@ -1065,7 +2201,15 @@ AdbcStatusCode AdbcStatementExecutePartitions(struct AdbcStatement *statement, s /// driver. /// /// Although drivers may choose any name for this function, the -/// recommended name is "AdbcDriverInit". +/// recommended name is "AdbcDriverInit", or a name derived from the +/// name of the driver's shared library as follows: remove the 'lib' +/// prefix (on Unix systems) and all file extensions, then PascalCase +/// the driver name, append Init, and prepend Adbc (if not already +/// there). For example: +/// +/// - libadbc_driver_sqlite.so.2.0.0 -> AdbcDriverSqliteInit +/// - adbc_driver_sqlite.dll -> AdbcDriverSqliteInit +/// - proprietary_driver.dll -> AdbcProprietaryDriverInit /// /// \param[in] version The ADBC revision to attempt to initialize (see /// ADBC_VERSION_1_0_0). diff --git a/src/duckdb/src/include/duckdb/common/adbc/adbc.hpp b/src/duckdb/src/include/duckdb/common/adbc/adbc.hpp index bc7e37fbd..c81b36473 100644 --- a/src/duckdb/src/include/duckdb/common/adbc/adbc.hpp +++ b/src/duckdb/src/include/duckdb/common/adbc/adbc.hpp @@ -33,8 +33,8 @@ AdbcStatusCode ConnectionInit(struct AdbcConnection *connection, struct AdbcData AdbcStatusCode ConnectionRelease(struct AdbcConnection *connection, struct AdbcError *error); -AdbcStatusCode ConnectionGetInfo(struct AdbcConnection *connection, uint32_t *info_codes, size_t info_codes_length, - struct ArrowArrayStream *out, struct AdbcError *error); +AdbcStatusCode ConnectionGetInfo(struct AdbcConnection *connection, const uint32_t *info_codes, + size_t info_codes_length, struct ArrowArrayStream *out, struct AdbcError *error); AdbcStatusCode ConnectionGetObjects(struct AdbcConnection *connection, int depth, const char *catalog, const char *db_schema, const char *table_name, const char **table_type, @@ -86,7 +86,7 @@ AdbcStatusCode StatementExecutePartitions(struct AdbcStatement *statement, struc //! This method should only be called when the string is guaranteed to not be NULL void SetError(struct AdbcError *error, const std::string &message); -void SetError(struct AdbcError *error, const char *message); +// void SetError(struct AdbcError *error, const char *message); void InitializeADBCError(AdbcError *error); diff --git a/src/duckdb/src/include/duckdb/common/adbc/options.h b/src/duckdb/src/include/duckdb/common/adbc/options.h new file mode 100644 index 000000000..6c9abb495 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/adbc/options.h @@ -0,0 +1,64 @@ +//////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////// +// THIS FILE IS GENERATED BY apache/arrow, DO NOT EDIT MANUALLY // +//////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////// + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +/// Common options that haven't yet been formally standardized. +/// https://github.com/apache/arrow-adbc/issues/1055 + +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/adbc/options.h +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#ifdef __cplusplus +extern "C" { +#endif + +/// \brief The catalog of the table for bulk insert. +/// +/// The type is char*. +#define ADBC_INGEST_OPTION_TARGET_CATALOG "adbc.ingest.target_catalog" + +/// \brief The schema of the table for bulk insert. +/// +/// The type is char*. +#define ADBC_INGEST_OPTION_TARGET_DB_SCHEMA "adbc.ingest.target_db_schema" + +/// \brief Use a temporary table for ingestion. +/// +/// The value should be ADBC_OPTION_VALUE_ENABLED or +/// ADBC_OPTION_VALUE_DISABLED (the default). +/// +/// This is not supported with ADBC_INGEST_OPTION_TARGET_CATALOG and +/// ADBC_INGEST_OPTION_TARGET_DB_SCHEMA. +/// +/// The type is char*. +#define ADBC_INGEST_OPTION_TEMPORARY "adbc.ingest.temporary" + +#ifdef __cplusplus +} +#endif diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/enum_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/enum_data.hpp index 087c622e8..ffcf729fb 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/enum_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/enum_data.hpp @@ -8,19 +8,14 @@ namespace duckdb { //===--------------------------------------------------------------------===// // Enums //===--------------------------------------------------------------------===// - -// FIXME: support Large offsets (int64_t), this does not currently respect the 'arrow_large_buffer_size' setting - template struct ArrowEnumData : public ArrowScalarBaseData { static idx_t GetLength(string_t input) { return input.GetSize(); } - static void WriteData(data_ptr_t target, string_t input) { memcpy(target, input.GetData(), input.GetSize()); } - static void EnumAppendVector(ArrowAppendData &append_data, const Vector &input, idx_t size) { D_ASSERT(input.GetVectorType() == VectorType::FLAT_VECTOR); @@ -28,9 +23,9 @@ struct ArrowEnumData : public ArrowScalarBaseData { ResizeValidity(append_data.validity, append_data.row_count + size); // resize the offset buffer - the offset buffer holds the offsets into the child array - append_data.main_buffer.resize(append_data.main_buffer.size() + sizeof(int32_t) * (size + 1)); + append_data.main_buffer.resize(append_data.main_buffer.size() + sizeof(uint32_t) * (size + 1)); auto data = FlatVector::GetData(input); - auto offset_data = append_data.main_buffer.GetData(); + auto offset_data = append_data.main_buffer.GetData(); if (append_data.row_count == 0) { // first entry offset_data[0] = 0; @@ -55,7 +50,6 @@ struct ArrowEnumData : public ArrowScalarBaseData { } append_data.row_count += size; } - static void Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity) { result.main_buffer.reserve(capacity * sizeof(TGT)); // construct the enum child data diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/list_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/list_data.hpp index 534ddef2a..9507ac729 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/list_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/list_data.hpp @@ -4,85 +4,15 @@ namespace duckdb { -template struct ArrowListData { public: - static void Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity) { - auto &child_type = ListType::GetChildType(type); - result.main_buffer.reserve((capacity + 1) * sizeof(BUFTYPE)); - auto child_buffer = ArrowAppender::InitializeChild(child_type, capacity, result.options); - result.child_data.push_back(std::move(child_buffer)); - } - - static void Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { - UnifiedVectorFormat format; - input.ToUnifiedFormat(input_size, format); - idx_t size = to - from; - vector child_indices; - AppendValidity(append_data, format, from, to); - AppendOffsets(append_data, format, from, to, child_indices); - - // append the child vector of the list - SelectionVector child_sel(child_indices.data()); - auto &child = ListVector::GetEntry(input); - auto child_size = child_indices.size(); - Vector child_copy(child.GetType()); - child_copy.Slice(child, child_sel, child_size); - append_data.child_data[0]->append_vector(*append_data.child_data[0], child_copy, 0, child_size, child_size); - append_data.row_count += size; - } - - static void Finalize(ArrowAppendData &append_data, const LogicalType &type, ArrowArray *result) { - result->n_buffers = 2; - result->buffers[1] = append_data.main_buffer.data(); - - auto &child_type = ListType::GetChildType(type); - ArrowAppender::AddChildren(append_data, 1); - result->children = append_data.child_pointers.data(); - result->n_children = 1; - append_data.child_arrays[0] = *ArrowAppender::FinalizeChild(child_type, std::move(append_data.child_data[0])); - } + static void Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity); + static void Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size); + static void Finalize(ArrowAppendData &append_data, const LogicalType &type, ArrowArray *result); public: static void AppendOffsets(ArrowAppendData &append_data, UnifiedVectorFormat &format, idx_t from, idx_t to, - vector &child_sel) { - // resize the offset buffer - the offset buffer holds the offsets into the child array - idx_t size = to - from; - append_data.main_buffer.resize(append_data.main_buffer.size() + sizeof(BUFTYPE) * (size + 1)); - auto data = UnifiedVectorFormat::GetData(format); - auto offset_data = append_data.main_buffer.GetData(); - if (append_data.row_count == 0) { - // first entry - offset_data[0] = 0; - } - // set up the offsets using the list entries - auto last_offset = offset_data[append_data.row_count]; - for (idx_t i = from; i < to; i++) { - auto source_idx = format.sel->get_index(i); - auto offset_idx = append_data.row_count + i + 1 - from; - - if (!format.validity.RowIsValid(source_idx)) { - offset_data[offset_idx] = last_offset; - continue; - } - - // append the offset data - auto list_length = data[source_idx].length; - if (std::is_same::value == true && - (uint64_t)last_offset + list_length > NumericLimits::Maximum()) { - throw InvalidInputException( - "Arrow Appender: The maximum combined list offset for regular list buffers is " - "%u but the offset of %lu exceeds this.", - NumericLimits::Maximum(), last_offset); - } - last_offset += list_length; - offset_data[offset_idx] = last_offset; - - for (idx_t k = 0; k < list_length; k++) { - child_sel.push_back(data[source_idx].offset + k); - } - } - } + vector &child_sel); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/map_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/map_data.hpp index e881c532a..9bb31c2fa 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/map_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/map_data.hpp @@ -2,96 +2,17 @@ #include "duckdb/common/arrow/arrow_appender.hpp" #include "duckdb/common/arrow/appender/append_data.hpp" -#include "duckdb/common/arrow/appender/list_data.hpp" namespace duckdb { //===--------------------------------------------------------------------===// // Maps //===--------------------------------------------------------------------===// -template struct ArrowMapData { public: - static void Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity) { - // map types are stored in a (too) clever way - // the main buffer holds the null values and the offsets - // then we have a single child, which is a struct of the map_type, and the key_type - result.main_buffer.reserve((capacity + 1) * sizeof(BUFTYPE)); - - auto &key_type = MapType::KeyType(type); - auto &value_type = MapType::ValueType(type); - auto internal_struct = make_uniq(result.options); - internal_struct->child_data.push_back(ArrowAppender::InitializeChild(key_type, capacity, result.options)); - internal_struct->child_data.push_back(ArrowAppender::InitializeChild(value_type, capacity, result.options)); - - result.child_data.push_back(std::move(internal_struct)); - } - - static void Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { - UnifiedVectorFormat format; - input.ToUnifiedFormat(input_size, format); - idx_t size = to - from; - AppendValidity(append_data, format, from, to); - vector child_indices; - ArrowListData::AppendOffsets(append_data, format, from, to, child_indices); - - SelectionVector child_sel(child_indices.data()); - auto &key_vector = MapVector::GetKeys(input); - auto &value_vector = MapVector::GetValues(input); - auto list_size = child_indices.size(); - - auto &struct_data = *append_data.child_data[0]; - auto &key_data = *struct_data.child_data[0]; - auto &value_data = *struct_data.child_data[1]; - - Vector key_vector_copy(key_vector.GetType()); - key_vector_copy.Slice(key_vector, child_sel, list_size); - Vector value_vector_copy(value_vector.GetType()); - value_vector_copy.Slice(value_vector, child_sel, list_size); - key_data.append_vector(key_data, key_vector_copy, 0, list_size, list_size); - value_data.append_vector(value_data, value_vector_copy, 0, list_size, list_size); - - append_data.row_count += size; - struct_data.row_count += size; - } - - static void Finalize(ArrowAppendData &append_data, const LogicalType &type, ArrowArray *result) { - // set up the main map buffer - D_ASSERT(result); - result->n_buffers = 2; - result->buffers[1] = append_data.main_buffer.data(); - - // the main map buffer has a single child: a struct - ArrowAppender::AddChildren(append_data, 1); - result->children = append_data.child_pointers.data(); - result->n_children = 1; - - auto &struct_data = *append_data.child_data[0]; - auto struct_result = ArrowAppender::FinalizeChild(type, std::move(append_data.child_data[0])); - - // Initialize the struct array data - const auto struct_child_count = 2; - ArrowAppender::AddChildren(struct_data, struct_child_count); - struct_result->children = struct_data.child_pointers.data(); - struct_result->n_buffers = 1; - struct_result->n_children = struct_child_count; - struct_result->length = struct_data.child_data[0]->row_count; - - append_data.child_arrays[0] = *struct_result; - - D_ASSERT(struct_data.child_data[0]->row_count == struct_data.child_data[1]->row_count); - - auto &key_type = MapType::KeyType(type); - auto &value_type = MapType::ValueType(type); - auto key_data = ArrowAppender::FinalizeChild(key_type, std::move(struct_data.child_data[0])); - struct_data.child_arrays[0] = *key_data; - struct_data.child_arrays[1] = *ArrowAppender::FinalizeChild(value_type, std::move(struct_data.child_data[1])); - - // keys cannot have null values - if (key_data->null_count > 0) { - throw std::runtime_error("Arrow doesn't accept NULL keys on Maps"); - } - } + static void Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity); + static void Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size); + static void Finalize(ArrowAppendData &append_data, const LogicalType &type, ArrowArray *result); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/varchar_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/varchar_data.hpp index fd2a2385d..03984fc77 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/varchar_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/varchar_data.hpp @@ -32,7 +32,7 @@ struct ArrowUUIDConverter { } }; -template +template struct ArrowVarcharData { static void Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity) { result.main_buffer.reserve((capacity + 1) * sizeof(BUFTYPE)); @@ -40,8 +40,7 @@ struct ArrowVarcharData { result.aux_buffer.reserve(capacity); } - template - static void AppendTemplated(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { + static void Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { idx_t size = to - from; UnifiedVectorFormat format; input.ToUnifiedFormat(input_size, format); @@ -61,6 +60,13 @@ struct ArrowVarcharData { // now append the string data to the auxiliary buffer // the auxiliary buffer's length depends on the string lengths, so we resize as required auto last_offset = offset_data[append_data.row_count]; + idx_t max_offset = append_data.row_count + to - from; + if (max_offset > NumericLimits::Maximum() && + append_data.options.arrow_offset_size == ArrowOffsetSize::REGULAR) { + throw InvalidInputException("Arrow Appender: The maximum total string size for regular string buffers is " + "%u but the offset of %lu exceeds this.", + NumericLimits::Maximum(), max_offset); + } for (idx_t i = from; i < to; i++) { auto source_idx = format.sel->get_index(i); auto offset_idx = append_data.row_count + i + 1 - from; @@ -78,13 +84,6 @@ struct ArrowVarcharData { // append the offset data auto current_offset = last_offset + string_length; - if (!LARGE_STRING && (int64_t)last_offset + string_length > NumericLimits::Maximum()) { - D_ASSERT(append_data.options.arrow_offset_size == ArrowOffsetSize::REGULAR); - throw InvalidInputException( - "Arrow Appender: The maximum total string size for regular string buffers is " - "%u but the offset of %lu exceeds this.", - NumericLimits::Maximum(), current_offset); - } offset_data[offset_idx] = current_offset; // resize the string buffer if required, and write the string data @@ -96,15 +95,6 @@ struct ArrowVarcharData { append_data.row_count += size; } - static void Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { - if (append_data.options.arrow_offset_size == ArrowOffsetSize::REGULAR) { - // Check if the offset exceeds the max supported value - AppendTemplated(append_data, input, from, to, input_size); - } else { - AppendTemplated(append_data, input, from, to, input_size); - } - } - static void Finalize(ArrowAppendData &append_data, const LogicalType &type, ArrowArray *result) { result->n_buffers = 3; result->buffers[1] = append_data.main_buffer.data(); diff --git a/src/duckdb/src/include/duckdb/common/enum_util.hpp b/src/duckdb/src/include/duckdb/common/enum_util.hpp index 3677a2fb1..6a6966c4d 100644 --- a/src/duckdb/src/include/duckdb/common/enum_util.hpp +++ b/src/duckdb/src/include/duckdb/common/enum_util.hpp @@ -208,8 +208,6 @@ enum class PreparedParamType : uint8_t; enum class ProfilerPrintFormat : uint8_t; -enum class QuantileSerializationType : uint8_t; - enum class QueryNodeType : uint8_t; enum class QueryResultType : uint8_t; @@ -238,8 +236,6 @@ enum class SinkCombineResultType : uint8_t; enum class SinkFinalizeType : uint8_t; -enum class SinkNextBatchType : uint8_t; - enum class SinkResultType : uint8_t; enum class SourceResultType : uint8_t; @@ -294,6 +290,10 @@ enum class WindowAggregationMode : uint32_t; enum class WindowBoundary : uint8_t; +enum class WindowExcludeMode : uint8_t; + +enum class WithinCollection : uint8_t; + template<> const char* EnumUtil::ToChars(AccessMode value); @@ -559,9 +559,6 @@ const char* EnumUtil::ToChars(PreparedParamType value); template<> const char* EnumUtil::ToChars(ProfilerPrintFormat value); -template<> -const char* EnumUtil::ToChars(QuantileSerializationType value); - template<> const char* EnumUtil::ToChars(QueryNodeType value); @@ -604,9 +601,6 @@ const char* EnumUtil::ToChars(SinkCombineResultType value template<> const char* EnumUtil::ToChars(SinkFinalizeType value); -template<> -const char* EnumUtil::ToChars(SinkNextBatchType value); - template<> const char* EnumUtil::ToChars(SinkResultType value); @@ -688,6 +682,12 @@ const char* EnumUtil::ToChars(WindowAggregationMode value template<> const char* EnumUtil::ToChars(WindowBoundary value); +template<> +const char* EnumUtil::ToChars(WindowExcludeMode value); + +template<> +const char* EnumUtil::ToChars(WithinCollection value); + template<> AccessMode EnumUtil::FromString(const char *value); @@ -953,9 +953,6 @@ PreparedParamType EnumUtil::FromString(const char *value); template<> ProfilerPrintFormat EnumUtil::FromString(const char *value); -template<> -QuantileSerializationType EnumUtil::FromString(const char *value); - template<> QueryNodeType EnumUtil::FromString(const char *value); @@ -998,9 +995,6 @@ SinkCombineResultType EnumUtil::FromString(const char *va template<> SinkFinalizeType EnumUtil::FromString(const char *value); -template<> -SinkNextBatchType EnumUtil::FromString(const char *value); - template<> SinkResultType EnumUtil::FromString(const char *value); @@ -1082,5 +1076,11 @@ WindowAggregationMode EnumUtil::FromString(const char *va template<> WindowBoundary EnumUtil::FromString(const char *value); +template<> +WindowExcludeMode EnumUtil::FromString(const char *value); + +template<> +WithinCollection EnumUtil::FromString(const char *value); + } diff --git a/src/duckdb/src/include/duckdb/common/enums/expression_type.hpp b/src/duckdb/src/include/duckdb/common/enums/expression_type.hpp index e404ef4ac..0b6516de0 100644 --- a/src/duckdb/src/include/duckdb/common/enums/expression_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/expression_type.hpp @@ -133,6 +133,7 @@ enum class ExpressionType : uint8_t { COLUMN_REF = 203, FUNCTION_REF = 204, TABLE_REF = 205, + LAMBDA_REF = 206, // ----------------------------- // Miscellaneous @@ -173,6 +174,7 @@ enum class ExpressionClass : uint8_t { LAMBDA = 17, POSITIONAL_REFERENCE = 18, BETWEEN = 19, + LAMBDA_REF = 20, //===--------------------------------------------------------------------===// // Bound Expressions //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/include/duckdb/common/enums/operator_result_type.hpp b/src/duckdb/src/include/duckdb/common/enums/operator_result_type.hpp index 5016ae2d2..f7ada047e 100644 --- a/src/duckdb/src/include/duckdb/common/enums/operator_result_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/operator_result_type.hpp @@ -56,10 +56,4 @@ enum class SinkCombineResultType : uint8_t { FINISHED, BLOCKED }; //! BLOCKED means the finalize call to the sink is currently blocked, e.g. by some async I/O. enum class SinkFinalizeType : uint8_t { READY, NO_OUTPUT_POSSIBLE, BLOCKED }; -//! The SinkNextBatchType is used to indicate the result of a NextBatch call on a sink -//! There are two possible results: -//! READY means the sink is ready for further processing -//! BLOCKED means the NextBatch call to the sink is currently blocked, e.g. by some async I/O. -enum class SinkNextBatchType : uint8_t { READY, BLOCKED }; - } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/extra_type_info.hpp b/src/duckdb/src/include/duckdb/common/extra_type_info.hpp index 74895fd01..d911d328b 100644 --- a/src/duckdb/src/include/duckdb/common/extra_type_info.hpp +++ b/src/duckdb/src/include/duckdb/common/extra_type_info.hpp @@ -23,7 +23,8 @@ enum class ExtraTypeInfoType : uint8_t { STRUCT_TYPE_INFO = 5, ENUM_TYPE_INFO = 6, USER_TYPE_INFO = 7, - AGGREGATE_STATE_TYPE_INFO = 8 + AGGREGATE_STATE_TYPE_INFO = 8, + ARRAY_TYPE_INFO = 9, }; struct ExtraTypeInfo { @@ -182,4 +183,17 @@ struct EnumTypeInfo : public ExtraTypeInfo { idx_t dict_size; }; +struct ArrayTypeInfo : public ExtraTypeInfo { + LogicalType child_type; + idx_t size; + explicit ArrayTypeInfo(LogicalType child_type_p, idx_t size_p); + +public: + void Serialize(Serializer &serializer) const override; + static shared_ptr Deserialize(Deserializer &reader); + +protected: + bool EqualsInternal(ExtraTypeInfo *other_p) const override; +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/file_system.hpp b/src/duckdb/src/include/duckdb/common/file_system.hpp index a9237d6fb..1f06b3ed6 100644 --- a/src/duckdb/src/include/duckdb/common/file_system.hpp +++ b/src/duckdb/src/include/duckdb/common/file_system.hpp @@ -66,7 +66,6 @@ struct FileHandle { DUCKDB_API string ReadLine(); DUCKDB_API bool CanSeek(); - DUCKDB_API bool IsPipe(); DUCKDB_API bool OnDiskFile(); DUCKDB_API idx_t GetFileSize(); DUCKDB_API FileType GetType(); diff --git a/src/duckdb/src/include/duckdb/common/multi_file_reader.hpp b/src/duckdb/src/include/duckdb/common/multi_file_reader.hpp index c83fae5fc..ca52810e8 100644 --- a/src/duckdb/src/include/duckdb/common/multi_file_reader.hpp +++ b/src/duckdb/src/include/duckdb/common/multi_file_reader.hpp @@ -9,11 +9,11 @@ #pragma once #include "duckdb/common/common.hpp" -#include "duckdb/common/multi_file_reader_options.hpp" #include "duckdb/common/enums/file_glob_options.hpp" -#include "duckdb/common/union_by_name.hpp" +#include "duckdb/common/multi_file_reader_options.hpp" #include "duckdb/common/optional_ptr.hpp" #include "duckdb/common/types/value.hpp" +#include "duckdb/common/union_by_name.hpp" namespace duckdb { class TableFunction; @@ -40,6 +40,8 @@ struct MultiFileReaderBindData { idx_t filename_idx = DConstants::INVALID_INDEX; //! The set of hive partitioning indexes (if any) vector hive_partitioning_indexes; + //! The index of the file_row_number column (if any) + idx_t file_row_number_idx = DConstants::INVALID_INDEX; DUCKDB_API void Serialize(Serializer &serializer) const; DUCKDB_API static MultiFileReaderBindData Deserialize(Deserializer &deserializer); @@ -110,6 +112,9 @@ struct MultiFileReader { const vector &global_names, const vector &global_column_ids, optional_ptr filters, MultiFileReaderData &reader_data, const string &initial_file); + //! Populated the filter_map + DUCKDB_API static void CreateFilterMap(const vector &global_types, + optional_ptr filters, MultiFileReaderData &reader_data); //! Finalize the reading of a chunk - applying any constants that are required DUCKDB_API static void FinalizeChunk(const MultiFileReaderBindData &bind_data, const MultiFileReaderData &reader_data, DataChunk &chunk); diff --git a/src/duckdb/src/include/duckdb/common/operator/cast_operators.hpp b/src/duckdb/src/include/duckdb/common/operator/cast_operators.hpp index fdcc90cd8..62fdb54f5 100644 --- a/src/duckdb/src/include/duckdb/common/operator/cast_operators.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/cast_operators.hpp @@ -580,27 +580,27 @@ template <> DUCKDB_API bool TryCastToTimestampSec::Operation(date_t input, timestamp_t &result, bool strict); //===--------------------------------------------------------------------===// -// Non-Standard Timestamps -> string/standard timestamp +// Non-Standard Timestamps -> string/timestamp types //===--------------------------------------------------------------------===// struct CastFromTimestampNS { template static inline string_t Operation(SRC input, Vector &result) { - throw duckdb::NotImplementedException("Cast to timestamp could not be performed!"); + throw duckdb::NotImplementedException("Cast to string could not be performed!"); } }; struct CastFromTimestampMS { template static inline string_t Operation(SRC input, Vector &result) { - throw duckdb::NotImplementedException("Cast to timestamp could not be performed!"); + throw duckdb::NotImplementedException("Cast to string could not be performed!"); } }; struct CastFromTimestampSec { template static inline string_t Operation(SRC input, Vector &result) { - throw duckdb::NotImplementedException("Cast to timestamp could not be performed!"); + throw duckdb::NotImplementedException("Cast to string could not be performed!"); } }; @@ -625,6 +625,13 @@ struct CastTimestampUsToSec { } }; +struct CastTimestampMsToDate { + template + static inline DST Operation(SRC input) { + throw duckdb::NotImplementedException("Cast to DATE could not be performed!"); + } +}; + struct CastTimestampMsToUs { template static inline DST Operation(SRC input) { @@ -639,6 +646,12 @@ struct CastTimestampMsToNs { } }; +struct CastTimestampNsToDate { + template + static inline DST Operation(SRC input) { + throw duckdb::NotImplementedException("Cast to DATE could not be performed!"); + } +}; struct CastTimestampNsToUs { template static inline DST Operation(SRC input) { @@ -646,6 +659,12 @@ struct CastTimestampNsToUs { } }; +struct CastTimestampSecToDate { + template + static inline DST Operation(SRC input) { + throw duckdb::NotImplementedException("Cast to DATE could not be performed!"); + } +}; struct CastTimestampSecToMs { template static inline DST Operation(SRC input) { @@ -667,19 +686,25 @@ struct CastTimestampSecToNs { } }; +template <> +duckdb::timestamp_t CastTimestampUsToSec::Operation(duckdb::timestamp_t input); template <> duckdb::timestamp_t CastTimestampUsToMs::Operation(duckdb::timestamp_t input); template <> duckdb::timestamp_t CastTimestampUsToNs::Operation(duckdb::timestamp_t input); template <> -duckdb::timestamp_t CastTimestampUsToSec::Operation(duckdb::timestamp_t input); +duckdb::date_t CastTimestampMsToDate::Operation(duckdb::timestamp_t input); template <> duckdb::timestamp_t CastTimestampMsToUs::Operation(duckdb::timestamp_t input); template <> duckdb::timestamp_t CastTimestampMsToNs::Operation(duckdb::timestamp_t input); template <> +duckdb::date_t CastTimestampNsToDate::Operation(duckdb::timestamp_t input); +template <> duckdb::timestamp_t CastTimestampNsToUs::Operation(duckdb::timestamp_t input); template <> +duckdb::date_t CastTimestampSecToDate::Operation(duckdb::timestamp_t input); +template <> duckdb::timestamp_t CastTimestampSecToMs::Operation(duckdb::timestamp_t input); template <> duckdb::timestamp_t CastTimestampSecToUs::Operation(duckdb::timestamp_t input); diff --git a/src/duckdb/src/include/duckdb/common/pipe_file_system.hpp b/src/duckdb/src/include/duckdb/common/pipe_file_system.hpp index 7806ce021..8d0502193 100644 --- a/src/duckdb/src/include/duckdb/common/pipe_file_system.hpp +++ b/src/duckdb/src/include/duckdb/common/pipe_file_system.hpp @@ -28,9 +28,6 @@ class PipeFileSystem : public FileSystem { bool CanSeek() override { return false; } - bool IsPipe(const string &filename) override { - return true; - } void FileSync(FileHandle &handle) override; std::string GetName() const override { diff --git a/src/duckdb/src/include/duckdb/common/serializer/deserializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/deserializer.hpp index 3014c51e9..33facb30e 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/deserializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/deserializer.hpp @@ -178,9 +178,9 @@ class Deserializer { return val; } - template - inline typename std::enable_if::value, T>::type Read() { - using ELEMENT_TYPE = typename is_unique_ptr::ELEMENT_TYPE; + // Deserialize unique_ptr if the element type has a Deserialize method + template ::ELEMENT_TYPE> + inline typename std::enable_if::value && has_deserialize::value, T>::type Read() { unique_ptr ptr = nullptr; auto is_present = OnNullableBegin(); if (is_present) { @@ -192,6 +192,20 @@ class Deserializer { return ptr; } + // Deserialize a unique_ptr if the element type does not have a Deserialize method + template ::ELEMENT_TYPE> + inline typename std::enable_if::value && !has_deserialize::value, T>::type Read() { + unique_ptr ptr = nullptr; + auto is_present = OnNullableBegin(); + if (is_present) { + OnObjectBegin(); + ptr = make_uniq(Read()); + OnObjectEnd(); + } + OnNullableEnd(); + return ptr; + } + // Deserialize shared_ptr template inline typename std::enable_if::value, T>::type Read() { diff --git a/src/duckdb/src/include/duckdb/common/sort/comparators.hpp b/src/duckdb/src/include/duckdb/common/sort/comparators.hpp index c05c16aa3..5f3cd3807 100644 --- a/src/duckdb/src/include/duckdb/common/sort/comparators.hpp +++ b/src/duckdb/src/include/duckdb/common/sort/comparators.hpp @@ -47,6 +47,8 @@ struct Comparators { //! Compares two struct values at the given pointers (recursive) static int CompareStructAndAdvance(data_ptr_t &left_ptr, data_ptr_t &right_ptr, const child_list_t &types, bool valid); + static int CompareArrayAndAdvance(data_ptr_t &left_ptr, data_ptr_t &right_ptr, const LogicalType &type, bool valid, + idx_t array_size); //! Compare two list values at the pointers (can be recursive if nested type) static int CompareListAndAdvance(data_ptr_t &left_ptr, data_ptr_t &right_ptr, const LogicalType &type, bool valid); //! Compares a list of fixed-size values diff --git a/src/duckdb/src/include/duckdb/common/string_util.hpp b/src/duckdb/src/include/duckdb/common/string_util.hpp index 7074387a6..e237ce4b8 100644 --- a/src/duckdb/src/include/duckdb/common/string_util.hpp +++ b/src/duckdb/src/include/duckdb/common/string_util.hpp @@ -11,6 +11,7 @@ #include "duckdb/common/constants.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/common/vector.hpp" +#include "duckdb/common/set.hpp" #include @@ -134,6 +135,7 @@ class StringUtil { //! Join multiple strings into one string. Components are concatenated by the given separator DUCKDB_API static string Join(const vector &input, const string &separator); + DUCKDB_API static string Join(const set &input, const string &separator); template static string ToString(const vector &input, const string &separator) { diff --git a/src/duckdb/src/include/duckdb/common/types.hpp b/src/duckdb/src/include/duckdb/common/types.hpp index a99bbd806..a9ba74dc9 100644 --- a/src/duckdb/src/include/duckdb/common/types.hpp +++ b/src/duckdb/src/include/duckdb/common/types.hpp @@ -151,8 +151,8 @@ enum class PhysicalType : uint8_t { ///// Custom data type, implemented by user //EXTENSION = 28, - ///// Fixed size list of some logical type - //FIXED_SIZE_LIST = 29, + ///// Array with fixed length of some logical type (a fixed-size list) + ARRAY = 29, ///// Measure of elapsed time in either seconds, milliseconds, microseconds ///// or nanoseconds. @@ -224,7 +224,8 @@ enum class LogicalTypeId : uint8_t { ENUM = 104, AGGREGATE_STATE = 105, LAMBDA = 106, - UNION = 107 + UNION = 107, + ARRAY = 108 }; @@ -364,6 +365,9 @@ struct LogicalType { DUCKDB_API static LogicalType MAP(const LogicalType &child); // NOLINT DUCKDB_API static LogicalType MAP(LogicalType key, LogicalType value); // NOLINT DUCKDB_API static LogicalType UNION( child_list_t members); // NOLINT + DUCKDB_API static LogicalType ARRAY(const LogicalType &child, idx_t size); // NOLINT + // an array of unknown size (only used for binding) + DUCKDB_API static LogicalType ARRAY(const LogicalType &child); // NOLINT DUCKDB_API static LogicalType ENUM(Vector &ordered_data, idx_t size); // NOLINT // DEPRECATED - provided for backwards compatibility DUCKDB_API static LogicalType ENUM(const string &enum_name, Vector &ordered_data, idx_t size); // NOLINT @@ -372,6 +376,8 @@ struct LogicalType { DUCKDB_API static const vector Numeric(); //! A list of all INTEGRAL types DUCKDB_API static const vector Integral(); + //! A list of all REAL types + DUCKDB_API static const vector Real(); //! A list of ALL SQL types DUCKDB_API static const vector AllTypes(); }; @@ -424,6 +430,13 @@ struct UnionType { DUCKDB_API static const child_list_t CopyMemberTypes(const LogicalType &type); }; +struct ArrayType { + DUCKDB_API static const LogicalType &GetChildType(const LogicalType &type); + DUCKDB_API static idx_t GetSize(const LogicalType &type); + DUCKDB_API static bool IsAnySize(const LogicalType &type); + DUCKDB_API static constexpr idx_t MAX_ARRAY_SIZE = 100000; //100k for now +}; + struct AggregateStateType { DUCKDB_API static const string GetTypeName(const LogicalType &type); DUCKDB_API static const aggregate_state_t &GetStateType(const LogicalType &type); diff --git a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_collection.hpp b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_collection.hpp index 34efa90d2..5c5aae636 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_collection.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_collection.hpp @@ -41,6 +41,12 @@ struct TupleDataGatherFunction { vector child_functions; }; +enum class WithinCollection : uint8_t { + NO, + LIST, + ARRAY, +}; + //! TupleDataCollection represents a set of buffer-managed data stored in row format //! FIXME: rename to RowDataCollection after we phase it out class TupleDataCollection { @@ -68,9 +74,11 @@ class TupleDataCollection { void Unpin(); //! Gets the scatter function for the given type - static TupleDataScatterFunction GetScatterFunction(const LogicalType &type, bool within_list = false); + static TupleDataScatterFunction GetScatterFunction(const LogicalType &type, + WithinCollection within = WithinCollection::NO); //! Gets the gather function for the given type - static TupleDataGatherFunction GetGatherFunction(const LogicalType &type, bool within_list = false); + static TupleDataGatherFunction GetGatherFunction(const LogicalType &type, + WithinCollection within = WithinCollection::NO); //! Initializes an Append state - useful for optimizing many appends made to the same tuple data collection void InitializeAppend(TupleDataAppendState &append_state, @@ -198,27 +206,30 @@ class TupleDataCollection { static void ComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, TupleDataVectorFormat &source, const SelectionVector &append_sel, const idx_t append_count); //! Computes the heap sizes for the specific Vector that will be appended (within a list) - static void WithinListHeapComputeSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, const SelectionVector &append_sel, - const idx_t append_count, const UnifiedVectorFormat &list_data); - //! Computes the heap sizes for the fixed-size type Vector that will be appended (within a list) - static void ComputeFixedWithinListHeapSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, const SelectionVector &append_sel, - const idx_t append_count, const UnifiedVectorFormat &list_data); - //! Computes the heap sizes for the string Vector that will be appended (within a list) - static void StringWithinListComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + static void WithinCollectionComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, TupleDataVectorFormat &source_format, const SelectionVector &append_sel, const idx_t append_count, const UnifiedVectorFormat &list_data); + //! Computes the heap sizes for the fixed-size type Vector that will be appended (within a list) + static void ComputeFixedWithinCollectionHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, const idx_t append_count, + const UnifiedVectorFormat &list_data); + //! Computes the heap sizes for the string Vector that will be appended (within a list) + static void StringWithinCollectionComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, const idx_t append_count, + const UnifiedVectorFormat &list_data); //! Computes the heap sizes for the struct Vector that will be appended (within a list) - static void StructWithinListComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, - const SelectionVector &append_sel, const idx_t append_count, - const UnifiedVectorFormat &list_data); + static void StructWithinCollectionComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, const idx_t append_count, + const UnifiedVectorFormat &list_data); //! Computes the heap sizes for the list Vector that will be appended (within a list) - static void ListWithinListComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, - TupleDataVectorFormat &source_format, const SelectionVector &append_sel, - const idx_t append_count, const UnifiedVectorFormat &list_data); + static void CollectionWithinCollectionComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, + TupleDataVectorFormat &source_format, + const SelectionVector &append_sel, const idx_t append_count, + const UnifiedVectorFormat &list_data); //! Get the next segment/chunk index for the scan bool NextScanIndex(TupleDataScanState &scan_state, idx_t &segment_index, idx_t &chunk_index); diff --git a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_states.hpp b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_states.hpp index 6d29d36c8..f2852d3b4 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_states.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_states.hpp @@ -45,6 +45,10 @@ struct TupleDataVectorFormat { UnifiedVectorFormat unified; vector children; unique_ptr combined_list_data; + + // Optional: only used for ArrayVector to fake being a list vector + unique_array array_list_entries; + idx_t parent_array_size; }; struct TupleDataChunkState { diff --git a/src/duckdb/src/include/duckdb/common/types/validity_mask.hpp b/src/duckdb/src/include/duckdb/common/types/validity_mask.hpp index d23c862ce..776ed535a 100644 --- a/src/duckdb/src/include/duckdb/common/types/validity_mask.hpp +++ b/src/duckdb/src/include/duckdb/common/types/validity_mask.hpp @@ -65,12 +65,11 @@ struct TemplatedValidityMask { static constexpr const int STANDARD_MASK_SIZE = STANDARD_ENTRY_COUNT * sizeof(validity_t); public: - inline TemplatedValidityMask() : validity_mask(nullptr) { + inline TemplatedValidityMask() : validity_mask(nullptr), target_count(STANDARD_VECTOR_SIZE) { } - inline explicit TemplatedValidityMask(idx_t max_count) { - Initialize(max_count); + inline explicit TemplatedValidityMask(idx_t target_count) : validity_mask(nullptr), target_count(target_count) { } - inline explicit TemplatedValidityMask(V *ptr) : validity_mask(ptr) { + inline explicit TemplatedValidityMask(V *ptr) : validity_mask(ptr), target_count(STANDARD_VECTOR_SIZE) { } inline TemplatedValidityMask(const TemplatedValidityMask &original, idx_t count) { Copy(original, count); @@ -139,6 +138,7 @@ struct TemplatedValidityMask { inline void Reset() { validity_mask = nullptr; validity_data.reset(); + target_count = STANDARD_VECTOR_SIZE; } static inline idx_t EntryCount(idx_t count) { @@ -229,8 +229,8 @@ struct TemplatedValidityMask { //! Marks the entry at the specified row index as invalid (i.e. null) inline void SetInvalid(idx_t row_idx) { if (!validity_mask) { - D_ASSERT(row_idx <= STANDARD_VECTOR_SIZE); - Initialize(STANDARD_VECTOR_SIZE); + D_ASSERT(row_idx <= target_count); + Initialize(target_count); } SetInvalidUnsafe(row_idx); } @@ -295,12 +295,18 @@ struct TemplatedValidityMask { inline void Initialize(const TemplatedValidityMask &other) { validity_mask = other.validity_mask; validity_data = other.validity_data; + target_count = other.target_count; } - inline void Initialize(idx_t count = STANDARD_VECTOR_SIZE) { + inline void Initialize(idx_t count) { + target_count = count; validity_data = make_buffer(count); validity_mask = validity_data->owned_data.get(); } + inline void Initialize() { + Initialize(target_count); + } inline void Copy(const TemplatedValidityMask &other, idx_t count) { + target_count = count; if (other.AllValid()) { validity_data = nullptr; validity_mask = nullptr; @@ -313,13 +319,15 @@ struct TemplatedValidityMask { protected: V *validity_mask; buffer_ptr validity_data; + // The size to initialize the validity mask to when/if the mask is lazily initialized + idx_t target_count; }; struct ValidityMask : public TemplatedValidityMask { public: inline ValidityMask() : TemplatedValidityMask(nullptr) { } - inline explicit ValidityMask(idx_t max_count) : TemplatedValidityMask(max_count) { + inline explicit ValidityMask(idx_t target_count) : TemplatedValidityMask(target_count) { } inline explicit ValidityMask(validity_t *ptr) : TemplatedValidityMask(ptr) { } @@ -328,7 +336,7 @@ struct ValidityMask : public TemplatedValidityMask { public: DUCKDB_API void Resize(idx_t old_size, idx_t new_size); - + DUCKDB_API idx_t TargetCount(); DUCKDB_API void SliceInPlace(const ValidityMask &other, idx_t target_offset, idx_t source_offset, idx_t count); DUCKDB_API void Slice(const ValidityMask &other, idx_t source_offset, idx_t count); DUCKDB_API void Combine(const ValidityMask &other, idx_t count); diff --git a/src/duckdb/src/include/duckdb/common/types/value.hpp b/src/duckdb/src/include/duckdb/common/types/value.hpp index 1d7481691..43948e21e 100644 --- a/src/duckdb/src/include/duckdb/common/types/value.hpp +++ b/src/duckdb/src/include/duckdb/common/types/value.hpp @@ -30,6 +30,7 @@ class Value { friend struct StructValue; friend struct ListValue; friend struct UnionValue; + friend struct ArrayValue; public: //! Create an empty NULL value of the specified type @@ -156,6 +157,13 @@ class Value { DUCKDB_API static Value LIST(const LogicalType &child_type, vector values); //! Create an empty list with the specified child-type DUCKDB_API static Value EMPTYLIST(const LogicalType &child_type); + //! Create an array value with the given entries. Array type is inferred from children + //! Cannot be called with an empty list, use either EMPTYARRAY or ARRAY with a type instead + DUCKDB_API static Value ARRAY(vector values); + // Create an array value with the given entries + DUCKDB_API static Value ARRAY(const LogicalType &type, vector values); + //! Create an empty array of the given type and size + DUCKDB_API static Value EMPTYARRAY(const LogicalType &type, uint32_t size); //! Create a map value with the given entries DUCKDB_API static Value MAP(const LogicalType &child_type, vector values); //! Create a union value from a selected value and a tag from a set of alternatives. @@ -391,6 +399,10 @@ struct ListValue { DUCKDB_API static const vector &GetChildren(const Value &value); }; +struct ArrayValue { + DUCKDB_API static const vector &GetChildren(const Value &value); +}; + struct UnionValue { DUCKDB_API static const Value &GetValue(const Value &value); DUCKDB_API static uint8_t GetTag(const Value &value); diff --git a/src/duckdb/src/include/duckdb/common/types/vector.hpp b/src/duckdb/src/include/duckdb/common/types/vector.hpp index 2ef4de986..1f6b56776 100644 --- a/src/duckdb/src/include/duckdb/common/types/vector.hpp +++ b/src/duckdb/src/include/duckdb/common/types/vector.hpp @@ -38,6 +38,7 @@ struct UnifiedVectorFormat { struct RecursiveUnifiedVectorFormat { UnifiedVectorFormat unified; vector children; + LogicalType logical_type; }; class VectorCache; @@ -65,6 +66,7 @@ class Vector { friend struct StructVector; friend struct UnionVector; friend struct SequenceVector; + friend struct ArrayVector; friend class DataChunk; friend class VectorCacheBuffer; @@ -447,6 +449,20 @@ struct StructVector { DUCKDB_API static vector> &GetEntries(Vector &vector); }; +struct ArrayVector { + //! Gets a reference to the underlying child-vector of an array + DUCKDB_API static const Vector &GetEntry(const Vector &vector); + //! Gets a reference to the underlying child-vector of an array + DUCKDB_API static Vector &GetEntry(Vector &vector); + //! Gets the total size of the underlying child-vector of an array + DUCKDB_API static idx_t GetTotalSize(const Vector &vector); + //! Allocate dummy list entries for a vector + //! Note that there is nothing ensuring that the allocated data + //! remains valid (e.g. if this vector is resized) + //! This is only used during row serialization + DUCKDB_API static void AllocateDummyListEntries(Vector &vector); +}; + enum class UnionInvalidReason : uint8_t { VALID, TAG_OUT_OF_RANGE, diff --git a/src/duckdb/src/include/duckdb/common/types/vector_buffer.hpp b/src/duckdb/src/include/duckdb/common/types/vector_buffer.hpp index 94341102b..3459b7c52 100644 --- a/src/duckdb/src/include/duckdb/common/types/vector_buffer.hpp +++ b/src/duckdb/src/include/duckdb/common/types/vector_buffer.hpp @@ -29,7 +29,8 @@ enum class VectorBufferType : uint8_t { STRUCT_BUFFER, // struct buffer, holds a ordered mapping from name to child vector LIST_BUFFER, // list buffer, holds a single flatvector child MANAGED_BUFFER, // managed buffer, holds a buffer managed by the buffermanager - OPAQUE_BUFFER // opaque buffer, can be created for example by the parquet reader + OPAQUE_BUFFER, // opaque buffer, can be created for example by the parquet reader + ARRAY_BUFFER // array buffer, holds a single flatvector child }; enum class VectorAuxiliaryDataType : uint8_t { @@ -271,6 +272,26 @@ class VectorListBuffer : public VectorBuffer { idx_t size = 0; }; +class VectorArrayBuffer : public VectorBuffer { +public: + explicit VectorArrayBuffer(unique_ptr child_vector, idx_t array_size, idx_t initial_capacity); + explicit VectorArrayBuffer(const LogicalType &array, idx_t initial = STANDARD_VECTOR_SIZE); + ~VectorArrayBuffer() override; + +public: + Vector &GetChild(); + idx_t GetArraySize(); + idx_t GetChildSize(); + +private: + unique_ptr child; + // The size of each array in this buffer + idx_t array_size = 0; + // How many arrays are currently stored in this buffer + // The child vector has size (array_size * size) + idx_t size = 0; +}; + //! The ManagedVectorBuffer holds a buffer handle class ManagedVectorBuffer : public VectorBuffer { public: diff --git a/src/duckdb/src/include/duckdb/common/vector_operations/aggregate_executor.hpp b/src/duckdb/src/include/duckdb/common/vector_operations/aggregate_executor.hpp index 4a80ac819..606276080 100644 --- a/src/duckdb/src/include/duckdb/common/vector_operations/aggregate_executor.hpp +++ b/src/duckdb/src/include/duckdb/common/vector_operations/aggregate_executor.hpp @@ -382,15 +382,74 @@ class AggregateExecutor { } } + using Frames = vector; + template static void UnaryWindow(Vector &input, const ValidityMask &ifilter, AggregateInputData &aggr_input_data, - data_ptr_t state, const FrameBounds &frame, const FrameBounds &prev, Vector &result, - idx_t rid, idx_t bias) { + data_ptr_t state_p, const Frames &frames, Vector &result, idx_t rid) { - auto idata = FlatVector::GetData(input) - bias; + auto idata = FlatVector::GetData(input); const auto &ivalid = FlatVector::Validity(input); - OP::template Window( - idata, ifilter, ivalid, aggr_input_data, *reinterpret_cast(state), frame, prev, result, rid, bias); + auto &state = *reinterpret_cast(state_p); + OP::template Window(idata, ifilter, ivalid, aggr_input_data, state, frames, + result, rid); + } + + template + static void IntersectFrames(const Frames &lefts, const Frames &rights, OP &op) { + const auto cover_start = MinValue(rights[0].start, lefts[0].start); + const auto cover_end = MaxValue(rights.back().end, lefts.back().end); + const FrameBounds last(cover_end, cover_end); + + // Subframe indices + idx_t l = 0; + idx_t r = 0; + for (auto i = cover_start; i < cover_end;) { + uint8_t overlap = 0; + + // Are we in the previous frame? + auto left = &last; + if (l < lefts.size()) { + left = &lefts[l]; + overlap |= uint8_t(left->start <= i && i < left->end) << 0; + } + + // Are we in the current frame? + auto right = &last; + if (r < rights.size()) { + right = &rights[r]; + overlap |= uint8_t(right->start <= i && i < right->end) << 1; + } + + auto limit = i; + switch (overlap) { + case 0x00: + // i ∉ F U P + limit = MinValue(right->start, left->start); + op.Neither(i, limit); + break; + case 0x01: + // i ∈ P \ F + limit = MinValue(left->end, right->start); + op.Left(i, limit); + break; + case 0x02: + // i ∈ F \ P + limit = MinValue(right->end, left->start); + op.Right(i, limit); + break; + case 0x03: + // i ∈ F ∩ P + limit = MinValue(right->end, left->end); + op.Both(i, limit); + break; + } + + // Advance the subframe indices + i = limit; + l += (i == left->end); + r += (i == right->end); + } } template diff --git a/src/duckdb/src/include/duckdb/core_functions/aggregate/distributive_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/aggregate/distributive_functions.hpp index 6a9185788..a454620a9 100644 --- a/src/duckdb/src/include/duckdb/core_functions/aggregate/distributive_functions.hpp +++ b/src/duckdb/src/include/duckdb/core_functions/aggregate/distributive_functions.hpp @@ -210,12 +210,6 @@ struct GroupConcatFun { static constexpr const char *Name = "group_concat"; }; -struct ListaggFun { - using ALIAS = StringAggFun; - - static constexpr const char *Name = "listagg"; -}; - struct SumFun { static constexpr const char *Name = "sum"; static constexpr const char *Parameters = "arg"; diff --git a/src/duckdb/src/include/duckdb/core_functions/aggregate/quantile_enum.hpp b/src/duckdb/src/include/duckdb/core_functions/aggregate/quantile_enum.hpp deleted file mode 100644 index 161f1a90e..000000000 --- a/src/duckdb/src/include/duckdb/core_functions/aggregate/quantile_enum.hpp +++ /dev/null @@ -1,21 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/core_functions/aggregate/quantile_enum.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -namespace duckdb { - -enum class QuantileSerializationType : uint8_t { - NON_DECIMAL = 0, - DECIMAL_DISCRETE, - DECIMAL_DISCRETE_LIST, - DECIMAL_CONTINUOUS, - DECIMAL_CONTINUOUS_LIST -}; - -} diff --git a/src/duckdb/src/include/duckdb/core_functions/lambda_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/lambda_functions.hpp new file mode 100644 index 000000000..6571f0f9e --- /dev/null +++ b/src/duckdb/src/include/duckdb/core_functions/lambda_functions.hpp @@ -0,0 +1,55 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/core_functions/lambda_functions.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/function/function.hpp" +#include "duckdb/execution/expression_executor_state.hpp" +#include "duckdb/execution/expression_executor.hpp" + +namespace duckdb { + +struct ListLambdaBindData : public FunctionData { +public: + ListLambdaBindData(const LogicalType &return_type, unique_ptr lambda_expr, const bool has_index = false) + : return_type(return_type), lambda_expr(std::move(lambda_expr)), has_index(has_index) {}; + + //! Return type of the scalar function + LogicalType return_type; + //! Lambda expression that the expression executor executes + unique_ptr lambda_expr; + //! True, if the last parameter in a lambda parameter list represents the index of the current list element + bool has_index; + +public: + bool Equals(const FunctionData &other_p) const override; + unique_ptr Copy() const override; + + //! Serializes a lambda function's bind data + static void Serialize(Serializer &serializer, const optional_ptr bind_data_p, + const ScalarFunction &function); + //! Deserializes a lambda function's bind data + static unique_ptr Deserialize(Deserializer &deserializer, ScalarFunction &); +}; + +class LambdaFunctions { +public: + //! Returns the parameter type for binary lambdas + static LogicalType BindBinaryLambda(const idx_t parameter_idx, const LogicalType &list_child_type); + //! Returns the ListLambdaBindData containing the lambda expression + static unique_ptr ListLambdaBind(ClientContext &, ScalarFunction &bound_function, + vector> &arguments, + const bool has_index = false); + + //! Internally executes list_transform + static void ListTransformFunction(DataChunk &args, ExpressionState &state, Vector &result); + //! Internally executes list_filter + static void ListFilterFunction(DataChunk &args, ExpressionState &state, Vector &result); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/core_functions/scalar/array_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/scalar/array_functions.hpp new file mode 100644 index 000000000..1cc099702 --- /dev/null +++ b/src/duckdb/src/include/duckdb/core_functions/scalar/array_functions.hpp @@ -0,0 +1,69 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/core_functions/scalar/array_functions.hpp +// +// +//===----------------------------------------------------------------------===// +// This file is automatically generated by scripts/generate_functions.py +// Do not edit this file manually, your changes will be overwritten +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/function/function_set.hpp" + +namespace duckdb { + +struct ArrayValueFun { + static constexpr const char *Name = "array_value"; + static constexpr const char *Parameters = "any,..."; + static constexpr const char *Description = "Create an ARRAY containing the argument values."; + static constexpr const char *Example = "array_value(4, 5, 6)"; + + static ScalarFunction GetFunction(); +}; + +struct ArrayCrossProductFun { + static constexpr const char *Name = "array_cross_product"; + static constexpr const char *Parameters = "array, array"; + static constexpr const char *Description = "Compute the cross product of two arrays of size 3. The array elements can not be NULL."; + static constexpr const char *Example = "array_cross_product([1, 2, 3], [1, 2, 3])"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ArrayCosineSimilarityFun { + static constexpr const char *Name = "array_cosine_similarity"; + static constexpr const char *Parameters = "array1,array2"; + static constexpr const char *Description = "Compute the cosine similarity between two arrays of the same size. The array elements can not be NULL. The arrays can have any size as long as the size is the same for both arguments."; + static constexpr const char *Example = "array_cosine_similarity([1, 2, 3], [1, 2, 3])"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ArrayDistanceFun { + static constexpr const char *Name = "array_distance"; + static constexpr const char *Parameters = "array1,array2"; + static constexpr const char *Description = "Compute the distance between two arrays of the same size. The array elements can not be NULL. The arrays can have any size as long as the size is the same for both arguments."; + static constexpr const char *Example = "array_distance([1, 2, 3], [1, 2, 3])"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ArrayInnerProductFun { + static constexpr const char *Name = "array_inner_product"; + static constexpr const char *Parameters = "array1,array2"; + static constexpr const char *Description = "Compute the inner product between two arrays of the same size. The array elements can not be NULL. The arrays can have any size as long as the size is the same for both arguments."; + static constexpr const char *Example = "array_inner_product([1, 2, 3], [1, 2, 3])"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ArrayDotProductFun { + using ALIAS = ArrayInnerProductFun; + + static constexpr const char *Name = "array_dot_product"; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/core_functions/scalar/list_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/scalar/list_functions.hpp index c1f2dc3e2..a3bf36976 100644 --- a/src/duckdb/src/include/duckdb/core_functions/scalar/list_functions.hpp +++ b/src/duckdb/src/include/duckdb/core_functions/scalar/list_functions.hpp @@ -132,6 +132,27 @@ struct ArraySortFun { static constexpr const char *Name = "array_sort"; }; +struct ListGradeUpFun { + static constexpr const char *Name = "list_grade_up"; + static constexpr const char *Parameters = "list"; + static constexpr const char *Description = "Returns the index of their sorted position."; + static constexpr const char *Example = "list_grade_up([3, 6, 1, 2])"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ArrayGradeUpFun { + using ALIAS = ListGradeUpFun; + + static constexpr const char *Name = "array_grade_up"; +}; + +struct GradeUpFun { + using ALIAS = ListGradeUpFun; + + static constexpr const char *Name = "grade_up"; +}; + struct ListReverseSortFun { static constexpr const char *Name = "list_reverse_sort"; static constexpr const char *Parameters = "list"; diff --git a/src/duckdb/src/include/duckdb/core_functions/scalar/math_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/scalar/math_functions.hpp index 367b360b3..771ffa17d 100644 --- a/src/duckdb/src/include/duckdb/core_functions/scalar/math_functions.hpp +++ b/src/duckdb/src/include/duckdb/core_functions/scalar/math_functions.hpp @@ -298,12 +298,9 @@ struct Log10Fun { }; struct LogFun { - static constexpr const char *Name = "log"; - static constexpr const char *Parameters = "b, x"; - static constexpr const char *Description = "Computes the logarithm of x to base b. b may be omitted, in which case the default 10"; - static constexpr const char *Example = "log(2, 64)"; + using ALIAS = Log10Fun; - static ScalarFunctionSet GetFunctions(); + static constexpr const char *Name = "log"; }; struct NextAfterFun { diff --git a/src/duckdb/src/include/duckdb/core_functions/to_interval.hpp b/src/duckdb/src/include/duckdb/core_functions/to_interval.hpp deleted file mode 100644 index 300b5fc3b..000000000 --- a/src/duckdb/src/include/duckdb/core_functions/to_interval.hpp +++ /dev/null @@ -1,30 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/core_functions/to_interval.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/function/function_set.hpp" -#include "duckdb/common/operator/multiply.hpp" - -namespace duckdb { - -struct ToSecondsOperator { - template - static inline TR Operation(TA input) { - interval_t result; - result.months = 0; - result.days = 0; - if (!TryMultiplyOperator::Operation(input, Interval::MICROS_PER_SEC, - result.micros)) { - throw OutOfRangeException("Interval value %d seconds out of range", input); - } - return result; - } -}; - -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_copy_to_file.hpp b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_copy_to_file.hpp index ff471a5df..e91e4d923 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_copy_to_file.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_copy_to_file.hpp @@ -46,7 +46,7 @@ class PhysicalBatchCopyToFile : public PhysicalOperator { OperatorSinkFinalizeInput &input) const override; unique_ptr GetLocalSinkState(ExecutionContext &context) const override; unique_ptr GetGlobalSinkState(ClientContext &context) const override; - SinkNextBatchType NextBatch(ExecutionContext &context, OperatorSinkNextBatchInput &input) const override; + void NextBatch(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate_p) const override; bool RequiresBatchIndex() const override { return true; diff --git a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_insert.hpp b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_insert.hpp index 55ec0a018..bc87f59f0 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_insert.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_insert.hpp @@ -52,7 +52,7 @@ class PhysicalBatchInsert : public PhysicalOperator { // Sink interface unique_ptr GetGlobalSinkState(ClientContext &context) const override; unique_ptr GetLocalSinkState(ExecutionContext &context) const override; - SinkNextBatchType NextBatch(ExecutionContext &context, OperatorSinkNextBatchInput &input) const override; + void NextBatch(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate_p) const override; SinkResultType Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const override; SinkCombineResultType Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const override; SinkFinalizeType Finalize(Pipeline &pipeline, Event &event, ClientContext &context, diff --git a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_fixed_batch_copy.hpp b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_fixed_batch_copy.hpp index 432234b9d..b8ca7329e 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_fixed_batch_copy.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_fixed_batch_copy.hpp @@ -45,7 +45,7 @@ class PhysicalFixedBatchCopy : public PhysicalOperator { OperatorSinkFinalizeInput &input) const override; unique_ptr GetLocalSinkState(ExecutionContext &context) const override; unique_ptr GetGlobalSinkState(ClientContext &context) const override; - SinkNextBatchType NextBatch(ExecutionContext &context, OperatorSinkNextBatchInput &input) const override; + void NextBatch(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate_p) const override; bool RequiresBatchIndex() const override { return true; diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp index b002c4685..9ad5288c0 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp @@ -131,7 +131,7 @@ struct CSVReaderOptions { //! Consider all columns to be of type varchar bool all_varchar = false; //! Whether or not to automatically detect dialect and datatypes - bool auto_detect = false; + bool auto_detect = true; //! The file path of the CSV file to read string file_path; //! Multi-file reader options @@ -202,9 +202,5 @@ struct CSVReaderOptions { string ToString() const; named_parameter_map_t OutputReadSettings(); - -public: - //! Whether columns were explicitly provided through named parameters - bool explicitly_set_columns = false; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp index 943c8a655..996248f9b 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp @@ -24,11 +24,55 @@ struct SnifferResult { vector names; }; +//! This represents the data related to columns that have been set by the user +//! e.g., from a copy command +struct SetColumns { + SetColumns(const vector *types_p, const vector *names_p) : types(types_p), names(names_p) { + if (!types) { + D_ASSERT(!types && !names); + } else { + D_ASSERT(types->size() == names->size()); + } + } + SetColumns() {}; + //! Return Types that were detected + const vector *types = nullptr; + //! Column Names that were detected + const vector *names = nullptr; + //! If columns are set + bool IsSet(); + //! How many columns + idx_t Size(); + //! Helper function that checks if candidate is acceptable based on the number of columns it produces + inline bool IsCandidateUnacceptable(idx_t num_cols, bool null_padding, bool ignore_errors) { + if (!IsSet() || ignore_errors) { + // We can't say its unacceptable if it's not set or if we ignore errors + return false; + } + idx_t size = Size(); + // If the columns are set and there is a mismatch with the expected number of columns, with null_padding and + // ignore_errors not set, we don't have a suitable candidate. + // Note that we compare with max_columns_found + 1, because some broken files have the behaviour where two + // columns are represented as: | col 1 | col_2 | + if (num_cols == size || num_cols == size + 1) { + // Good Candidate + return false; + } + // if we detected more columns than we have set, it's all good because we can null-pad them + if (null_padding && num_cols > size) { + return false; + } + + // Unacceptable + return true; + } +}; + //! Sniffer that detects Header, Dialect and Types of CSV Files class CSVSniffer { public: explicit CSVSniffer(CSVReaderOptions &options_p, shared_ptr buffer_manager_p, - CSVStateMachineCache &state_machine_cache, bool explicit_set_columns = false); + CSVStateMachineCache &state_machine_cache, SetColumns set_columns = {}); //! Main method that sniffs the CSV file, returns the types, names and options as a result //! CSV Sniffing consists of five steps: @@ -50,6 +94,8 @@ class CSVSniffer { CSVReaderOptions &options; //! Buffer being used on sniffer shared_ptr buffer_manager; + //! Information regarding columns that were set by user/query + SetColumns set_columns; //! Sets the result options void SetResultOptions(); @@ -119,8 +165,6 @@ class CSVSniffer { //! ------------------------------------------------------// void DetectHeader(); vector names; - //! If Column Names and Types have been explicitly set - const bool explicit_set_columns; //! ------------------------------------------------------// //! ------------------ Type Replacement ----------------- // diff --git a/src/duckdb/src/include/duckdb/execution/physical_operator.hpp b/src/duckdb/src/include/duckdb/execution/physical_operator.hpp index 076e5f5e7..e1304673c 100644 --- a/src/duckdb/src/include/duckdb/execution/physical_operator.hpp +++ b/src/duckdb/src/include/duckdb/execution/physical_operator.hpp @@ -145,8 +145,8 @@ class PhysicalOperator { virtual SinkFinalizeType Finalize(Pipeline &pipeline, Event &event, ClientContext &context, OperatorSinkFinalizeInput &input) const; //! For sinks with RequiresBatchIndex set to true, when a new batch starts being processed this method is called - //! This allows flushing of the current batch (e.g. to disk) - virtual SinkNextBatchType NextBatch(ExecutionContext &context, OperatorSinkNextBatchInput &input) const; + //! This allows flushing of the current batch (e.g. to disk) TODO: should this be able to block too? + virtual void NextBatch(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate_p) const; virtual unique_ptr GetLocalSinkState(ExecutionContext &context) const; virtual unique_ptr GetGlobalSinkState(ClientContext &context) const; diff --git a/src/duckdb/src/include/duckdb/execution/physical_operator_states.hpp b/src/duckdb/src/include/duckdb/execution/physical_operator_states.hpp index d799ec0d7..e3963efbe 100644 --- a/src/duckdb/src/include/duckdb/execution/physical_operator_states.hpp +++ b/src/duckdb/src/include/duckdb/execution/physical_operator_states.hpp @@ -176,12 +176,6 @@ struct OperatorSinkFinalizeInput { InterruptState &interrupt_state; }; -struct OperatorSinkNextBatchInput { - GlobalSinkState &global_state; - LocalSinkState &local_state; - InterruptState &interrupt_state; -}; - // LCOV_EXCL_STOP } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/window_executor.hpp b/src/duckdb/src/include/duckdb/execution/window_executor.hpp index beadd3d68..f7f151379 100644 --- a/src/duckdb/src/include/duckdb/execution/window_executor.hpp +++ b/src/duckdb/src/include/duckdb/execution/window_executor.hpp @@ -257,6 +257,7 @@ class WindowValueExecutor : public WindowExecutor { const ValidityMask &partition_mask, const ValidityMask &order_mask); void Sink(DataChunk &input_chunk, const idx_t input_idx, const idx_t total_count) override; + unique_ptr GetExecutorState() const override; protected: // IGNORE NULLS diff --git a/src/duckdb/src/include/duckdb/execution/window_segment_tree.hpp b/src/duckdb/src/include/duckdb/execution/window_segment_tree.hpp index 12786a19c..0cfce8af5 100644 --- a/src/duckdb/src/include/duckdb/execution/window_segment_tree.hpp +++ b/src/duckdb/src/include/duckdb/execution/window_segment_tree.hpp @@ -13,6 +13,7 @@ #include "duckdb/function/aggregate_function.hpp" #include "duckdb/common/enums/window_aggregation_mode.hpp" #include "duckdb/execution/operator/aggregate/aggregate_object.hpp" +#include "duckdb/parser/expression/window_expression.hpp" namespace duckdb { @@ -39,7 +40,8 @@ class WindowAggregatorState { class WindowAggregator { public: - WindowAggregator(AggregateObject aggr, const LogicalType &result_type_p, idx_t partition_count); + WindowAggregator(AggregateObject aggr, const LogicalType &result_type_p, const WindowExcludeMode exclude_mode_p, + idx_t partition_count); virtual ~WindowAggregator(); // Build @@ -48,8 +50,8 @@ class WindowAggregator { // Probe virtual unique_ptr GetLocalState() const = 0; - virtual void Evaluate(WindowAggregatorState &lstate, const idx_t *begins, const idx_t *ends, Vector &result, - idx_t count) const = 0; + virtual void Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, idx_t count, + idx_t row_idx) const = 0; protected: AggregateObject aggr; @@ -69,12 +71,16 @@ class WindowAggregator { idx_t filter_pos; //! The state used by the aggregator to build. unique_ptr gstate; + +public: + //! The window exclusion clause + const WindowExcludeMode exclude_mode; }; class WindowConstantAggregator : public WindowAggregator { public: WindowConstantAggregator(AggregateObject aggr, const LogicalType &result_type_p, const ValidityMask &partition_mask, - const idx_t count); + WindowExcludeMode exclude_mode_p, const idx_t count); ~WindowConstantAggregator() override { } @@ -82,8 +88,8 @@ class WindowConstantAggregator : public WindowAggregator { void Finalize() override; unique_ptr GetLocalState() const override; - void Evaluate(WindowAggregatorState &lstate, const idx_t *begins, const idx_t *ends, Vector &result, - idx_t count) const override; + void Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, idx_t count, + idx_t row_idx) const override; private: void AggregateInit(); @@ -107,24 +113,27 @@ class WindowConstantAggregator : public WindowAggregator { class WindowCustomAggregator : public WindowAggregator { public: - WindowCustomAggregator(AggregateObject aggr, const LogicalType &result_type_p, idx_t partition_count); + WindowCustomAggregator(AggregateObject aggr, const LogicalType &result_type_p, + const WindowExcludeMode exclude_mode_p, idx_t partition_count); ~WindowCustomAggregator() override; unique_ptr GetLocalState() const override; - void Evaluate(WindowAggregatorState &lstate, const idx_t *begins, const idx_t *ends, Vector &result, - idx_t count) const override; + void Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, idx_t count, + idx_t row_idx) const override; }; class WindowSegmentTree : public WindowAggregator { + public: - WindowSegmentTree(AggregateObject aggr, const LogicalType &result_type, idx_t count, WindowAggregationMode mode_p); + WindowSegmentTree(AggregateObject aggr, const LogicalType &result_type, WindowAggregationMode mode_p, + const WindowExcludeMode exclude_mode_p, idx_t count); ~WindowSegmentTree() override; void Finalize() override; unique_ptr GetLocalState() const override; - void Evaluate(WindowAggregatorState &lstate, const idx_t *begins, const idx_t *ends, Vector &result, - idx_t count) const override; + void Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, idx_t count, + idx_t row_idx) const override; public: void ConstructTree(); diff --git a/src/duckdb/src/include/duckdb/function/aggregate_function.hpp b/src/duckdb/src/include/duckdb/function/aggregate_function.hpp index 56b712a31..7b23a1ab9 100644 --- a/src/duckdb/src/include/duckdb/function/aggregate_function.hpp +++ b/src/duckdb/src/include/duckdb/function/aggregate_function.hpp @@ -43,8 +43,7 @@ typedef void (*aggregate_simple_update_t)(Vector inputs[], AggregateInputData &a //! The type used for updating complex windowed aggregate functions (optional) typedef void (*aggregate_window_t)(Vector inputs[], const ValidityMask &filter_mask, AggregateInputData &aggr_input_data, idx_t input_count, data_ptr_t state, - const FrameBounds &frame, const FrameBounds &prev, Vector &result, idx_t rid, - idx_t bias); + const vector &frames, Vector &result, idx_t rid); typedef void (*aggregate_serialize_t)(Serializer &serializer, const optional_ptr bind_data, const AggregateFunction &function); @@ -219,11 +218,11 @@ class AggregateFunction : public BaseScalarFunction { template static void UnaryWindow(Vector inputs[], const ValidityMask &filter_mask, AggregateInputData &aggr_input_data, - idx_t input_count, data_ptr_t state, const FrameBounds &frame, const FrameBounds &prev, - Vector &result, idx_t rid, idx_t bias) { + idx_t input_count, data_ptr_t state, const vector &frames, Vector &result, + idx_t rid) { D_ASSERT(input_count == 1); AggregateExecutor::UnaryWindow(inputs[0], filter_mask, aggr_input_data, - state, frame, prev, result, rid, bias); + state, frames, result, rid); } template diff --git a/src/duckdb/src/include/duckdb/function/cast/bound_cast_data.hpp b/src/duckdb/src/include/duckdb/function/cast/bound_cast_data.hpp index 9a81dfe6e..8645c24dd 100644 --- a/src/duckdb/src/include/duckdb/function/cast/bound_cast_data.hpp +++ b/src/duckdb/src/include/duckdb/function/cast/bound_cast_data.hpp @@ -27,6 +27,22 @@ struct ListBoundCastData : public BoundCastData { } }; +struct ArrayBoundCastData : public BoundCastData { + explicit ArrayBoundCastData(BoundCastInfo child_cast) : child_cast_info(std::move(child_cast)) { + } + + BoundCastInfo child_cast_info; + + static unique_ptr BindArrayToArrayCast(BindCastInput &input, const LogicalType &source, + const LogicalType &target); + static unique_ptr InitArrayLocalState(CastLocalStateParameters ¶meters); + +public: + unique_ptr Copy() const override { + return make_uniq(child_cast_info.Copy()); + } +}; + struct ListCast { static bool ListToListCast(Vector &source, Vector &result, idx_t count, CastParameters ¶meters); }; diff --git a/src/duckdb/src/include/duckdb/function/cast/default_casts.hpp b/src/duckdb/src/include/duckdb/function/cast/default_casts.hpp index 97e9ff902..a102f08ad 100644 --- a/src/duckdb/src/include/duckdb/function/cast/default_casts.hpp +++ b/src/duckdb/src/include/duckdb/function/cast/default_casts.hpp @@ -135,6 +135,7 @@ struct DefaultCasts { static BoundCastInfo EnumCastSwitch(BindCastInput &input, const LogicalType &source, const LogicalType &target); static BoundCastInfo IntervalCastSwitch(BindCastInput &input, const LogicalType &source, const LogicalType &target); static BoundCastInfo ListCastSwitch(BindCastInput &input, const LogicalType &source, const LogicalType &target); + static BoundCastInfo ArrayCastSwitch(BindCastInput &input, const LogicalType &source, const LogicalType &target); static BoundCastInfo NumericCastSwitch(BindCastInput &input, const LogicalType &source, const LogicalType &target); static BoundCastInfo MapCastSwitch(BindCastInput &input, const LogicalType &source, const LogicalType &target); static BoundCastInfo PointerCastSwitch(BindCastInput &input, const LogicalType &source, const LogicalType &target); diff --git a/src/duckdb/src/include/duckdb/function/cast/vector_cast_helpers.hpp b/src/duckdb/src/include/duckdb/function/cast/vector_cast_helpers.hpp index c7138f971..05911da6c 100644 --- a/src/duckdb/src/include/duckdb/function/cast/vector_cast_helpers.hpp +++ b/src/duckdb/src/include/duckdb/function/cast/vector_cast_helpers.hpp @@ -206,6 +206,12 @@ struct VectorStringToList { const SelectionVector *sel); }; +struct VectorStringToArray { + static bool StringToNestedTypeCastLoop(const string_t *source_data, ValidityMask &source_mask, Vector &result, + ValidityMask &result_mask, idx_t count, CastParameters ¶meters, + const SelectionVector *sel); +}; + struct VectorStringToStruct { static bool SplitStruct(const string_t &input, vector> &varchar_vectors, idx_t &row_idx, string_map_t &child_names, vector &child_masks); diff --git a/src/duckdb/src/include/duckdb/function/scalar/nested_functions.hpp b/src/duckdb/src/include/duckdb/function/scalar/nested_functions.hpp index 8fc34a3ce..ed51471e1 100644 --- a/src/duckdb/src/include/duckdb/function/scalar/nested_functions.hpp +++ b/src/duckdb/src/include/duckdb/function/scalar/nested_functions.hpp @@ -103,6 +103,21 @@ struct ListResizeFun { static void RegisterFunction(BuiltinFunctions &set); }; +struct ListZipFun { + static ScalarFunction GetFunction(); + static void RegisterFunction(BuiltinFunctions &set); +}; + +struct ListSelectFun { + static ScalarFunction GetFunction(); + static void RegisterFunction(BuiltinFunctions &set); +}; + +struct ListWhereFun { + static ScalarFunction GetFunction(); + static void RegisterFunction(BuiltinFunctions &set); +}; + struct StructExtractFun { static ScalarFunction GetFunction(); static void RegisterFunction(BuiltinFunctions &set); diff --git a/src/duckdb/src/include/duckdb/function/scalar_function.hpp b/src/duckdb/src/include/duckdb/function/scalar_function.hpp index 79b5a87d4..caa4cc634 100644 --- a/src/duckdb/src/include/duckdb/function/scalar_function.hpp +++ b/src/duckdb/src/include/duckdb/function/scalar_function.hpp @@ -51,17 +51,21 @@ struct FunctionStatisticsInput { unique_ptr *expr_ptr; }; -//! The type used for scalar functions +//! The scalar function type typedef std::function scalar_function_t; -//! Binds the scalar function and creates the function data +//! The type to bind the scalar function and to create the function data typedef unique_ptr (*bind_scalar_function_t)(ClientContext &context, ScalarFunction &bound_function, vector> &arguments); +//! The type to initialize a thread local state for the scalar function typedef unique_ptr (*init_local_state_t)(ExpressionState &state, const BoundFunctionExpression &expr, FunctionData *bind_data); -typedef unique_ptr (*function_statistics_t)(ClientContext &context, FunctionStatisticsInput &input); -//! Adds the dependencies of this BoundFunctionExpression to the set of dependencies +//! The type to add the dependencies of this BoundFunctionExpression to the set of dependencies typedef void (*dependency_function_t)(BoundFunctionExpression &expr, DependencyList &dependencies); +//! The type to propagate statistics for this scalar function +typedef unique_ptr (*function_statistics_t)(ClientContext &context, FunctionStatisticsInput &input); +//! The type to bind lambda-specific parameter types +typedef LogicalType (*bind_lambda_function_t)(const idx_t parameter_idx, const LogicalType &list_child_type); typedef void (*function_serialize_t)(Serializer &serializer, const optional_ptr bind_data, const ScalarFunction &function); @@ -75,14 +79,16 @@ class ScalarFunction : public BaseScalarFunction { init_local_state_t init_local_state = nullptr, LogicalType varargs = LogicalType(LogicalTypeId::INVALID), FunctionSideEffects side_effects = FunctionSideEffects::NO_SIDE_EFFECTS, - FunctionNullHandling null_handling = FunctionNullHandling::DEFAULT_NULL_HANDLING); + FunctionNullHandling null_handling = FunctionNullHandling::DEFAULT_NULL_HANDLING, + bind_lambda_function_t bind_lambda = nullptr); DUCKDB_API ScalarFunction(vector arguments, LogicalType return_type, scalar_function_t function, bind_scalar_function_t bind = nullptr, dependency_function_t dependency = nullptr, function_statistics_t statistics = nullptr, init_local_state_t init_local_state = nullptr, LogicalType varargs = LogicalType(LogicalTypeId::INVALID), FunctionSideEffects side_effects = FunctionSideEffects::NO_SIDE_EFFECTS, - FunctionNullHandling null_handling = FunctionNullHandling::DEFAULT_NULL_HANDLING); + FunctionNullHandling null_handling = FunctionNullHandling::DEFAULT_NULL_HANDLING, + bind_lambda_function_t bind_lambda = nullptr); //! The main scalar function to execute scalar_function_t function; @@ -94,6 +100,8 @@ class ScalarFunction : public BaseScalarFunction { dependency_function_t dependency; //! The statistics propagation function (if any) function_statistics_t statistics; + //! The lambda bind function (if any) + bind_lambda_function_t bind_lambda; function_serialize_t serialize; function_deserialize_t deserialize; diff --git a/src/duckdb/src/include/duckdb/function/table/system_functions.hpp b/src/duckdb/src/include/duckdb/function/table/system_functions.hpp index 8cada10fb..f6ff53088 100644 --- a/src/duckdb/src/include/duckdb/function/table/system_functions.hpp +++ b/src/duckdb/src/include/duckdb/function/table/system_functions.hpp @@ -133,8 +133,4 @@ struct TestVectorTypesFun { static void RegisterFunction(BuiltinFunctions &set); }; -struct PragmaUserAgent { - static void RegisterFunction(BuiltinFunctions &set); -}; - } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/function/udf_function.hpp b/src/duckdb/src/include/duckdb/function/udf_function.hpp index a679d4cec..655692930 100644 --- a/src/duckdb/src/include/duckdb/function/udf_function.hpp +++ b/src/duckdb/src/include/duckdb/function/udf_function.hpp @@ -370,7 +370,7 @@ struct UDFWrapper { inline static AggregateFunction CreateBinaryAggregateFunction(const string &name, LogicalType ret_type, LogicalType input_typeA, LogicalType input_typeB) { AggregateFunction aggr_function = - AggregateFunction::BinaryAggregate(input_typeA, input_typeB, ret_type); + AggregateFunction::BinaryAggregate(input_typeA, input_typeB, ret_type); aggr_function.name = name; return aggr_function; } diff --git a/src/duckdb/src/include/duckdb/main/capi/capi_internal.hpp b/src/duckdb/src/include/duckdb/main/capi/capi_internal.hpp index cd4ae6262..3a9e9a674 100644 --- a/src/duckdb/src/include/duckdb/main/capi/capi_internal.hpp +++ b/src/duckdb/src/include/duckdb/main/capi/capi_internal.hpp @@ -49,7 +49,6 @@ struct PendingStatementWrapper { struct ArrowResultWrapper { unique_ptr result; unique_ptr current_chunk; - ClientProperties options; }; struct AppenderWrapper { diff --git a/src/duckdb/src/include/duckdb/main/config.hpp b/src/duckdb/src/include/duckdb/main/config.hpp index 3cd2e8850..654bc80c7 100644 --- a/src/duckdb/src/include/duckdb/main/config.hpp +++ b/src/duckdb/src/include/duckdb/main/config.hpp @@ -173,10 +173,6 @@ struct DBConfigOptions { static bool debug_print_bindings; //! The peak allocation threshold at which to flush the allocator after completing a task (1 << 27, ~128MB) idx_t allocator_flush_threshold = 134217728; - //! DuckDB API surface - string duckdb_api; - //! Metadata from DuckDB callers - string custom_user_agent; bool operator==(const DBConfigOptions &other) const; }; @@ -263,7 +259,6 @@ struct DBConfig { OrderType ResolveOrder(OrderType order_type) const; OrderByNullType ResolveNullOrder(OrderType order_type, OrderByNullType null_type) const; - const std::string UserAgent() const; private: unique_ptr compression_functions; diff --git a/src/duckdb/src/include/duckdb/main/connection.hpp b/src/duckdb/src/include/duckdb/main/connection.hpp index 29048a9e4..75cae1500 100644 --- a/src/duckdb/src/include/duckdb/main/connection.hpp +++ b/src/duckdb/src/include/duckdb/main/connection.hpp @@ -131,7 +131,8 @@ class Connection { //! Reads CSV file DUCKDB_API shared_ptr ReadCSV(const string &csv_file); - DUCKDB_API shared_ptr ReadCSV(const string &csv_file, named_parameter_map_t &&options); + DUCKDB_API shared_ptr ReadCSV(const vector &csv_input, named_parameter_map_t &&options); + DUCKDB_API shared_ptr ReadCSV(const string &csv_input, named_parameter_map_t &&options); DUCKDB_API shared_ptr ReadCSV(const string &csv_file, const vector &columns); //! Reads Parquet file diff --git a/src/duckdb/src/include/duckdb/main/extension_entries.hpp b/src/duckdb/src/include/duckdb/main/extension_entries.hpp index 041c72e28..6804a2547 100644 --- a/src/duckdb/src/include/duckdb/main/extension_entries.hpp +++ b/src/duckdb/src/include/duckdb/main/extension_entries.hpp @@ -61,6 +61,7 @@ static constexpr ExtensionEntry EXTENSION_FUNCTIONS[] = { {"json_merge_patch", "json"}, {"json_object", "json"}, {"json_quote", "json"}, + {"json_serialize_plan", "json"}, {"json_serialize_sql", "json"}, {"json_structure", "json"}, {"json_transform", "json"}, @@ -69,13 +70,12 @@ static constexpr ExtensionEntry EXTENSION_FUNCTIONS[] = { {"json_valid", "json"}, {"load_aws_credentials", "aws"}, {"make_timestamptz", "icu"}, + {"parquet_kv_metadata", "parquet"}, {"parquet_metadata", "parquet"}, {"parquet_scan", "parquet"}, {"parquet_schema", "parquet"}, - {"pg_clear_cache", "postgres_scanner"}, {"pg_timezone_names", "icu"}, {"postgres_attach", "postgres_scanner"}, - {"postgres_query", "postgres_scanner"}, {"postgres_scan", "postgres_scanner"}, {"postgres_scan_pushdown", "postgres_scanner"}, {"read_json", "json"}, @@ -116,7 +116,6 @@ static constexpr ExtensionEntry EXTENSION_FUNCTIONS[] = { {"st_distance", "spatial"}, {"st_distance_spheroid", "spatial"}, {"st_drivers", "spatial"}, - {"st_dump", "spatial"}, {"st_dwithin", "spatial"}, {"st_dwithin_spheroid", "spatial"}, {"st_endpoint", "spatial"}, @@ -143,12 +142,9 @@ static constexpr ExtensionEntry EXTENSION_FUNCTIONS[] = { {"st_isvalid", "spatial"}, {"st_length", "spatial"}, {"st_length_spheroid", "spatial"}, - {"st_linemerge", "spatial"}, {"st_linestring2dfromwkb", "spatial"}, {"st_list_proj_crs", "spatial"}, - {"st_makeenvelope", "spatial"}, {"st_makeline", "spatial"}, - {"st_makepolygon", "spatial"}, {"st_ngeometries", "spatial"}, {"st_ninteriorrings", "spatial"}, {"st_normalize", "spatial"}, @@ -210,12 +206,6 @@ static constexpr ExtensionEntry EXTENSION_SETTINGS[] = { {"http_retry_backoff", "httpfs"}, {"http_retry_wait_ms", "httpfs"}, {"http_timeout", "httpfs"}, - {"pg_debug_show_queries", "postgres_scanner"}, - {"pg_use_binary_copy", "postgres_scanner"}, - {"pg_experimental_filter_pushdown", "postgres_scanner"}, - {"pg_connection_limit", "postgres_scanner"}, - {"pg_pages_per_task", "postgres_scanner"}, - {"pg_array_as_varchar", "postgres_scanner"}, {"s3_access_key_id", "httpfs"}, {"s3_endpoint", "httpfs"}, {"s3_region", "httpfs"}, @@ -270,9 +260,12 @@ static constexpr ExtensionEntry EXTENSION_COLLATIONS[] = { // Note: these are currently hardcoded in scripts/generate_extensions_function.py // TODO: automate by passing though to script via duckdb static constexpr ExtensionEntry EXTENSION_FILE_PREFIXES[] = { - {"http://", "httpfs"}, {"https://", "httpfs"}, {"s3://", "httpfs"}, - // {"azure://", "azure"} -}; // END_OF_EXTENSION_FILE_PREFIXES + {"http://", "httpfs"}, + {"https://", "httpfs"}, + {"s3://", "httpfs"}, + {"gcs://", "httpfs"}, + {"r2://", "httpfs"} // , {"azure://", "azure"} +}; // END_OF_EXTENSION_FILE_PREFIXES // Note: these are currently hardcoded in scripts/generate_extensions_function.py // TODO: automate by passing though to script via duckdb diff --git a/src/duckdb/src/include/duckdb/main/relation/read_csv_relation.hpp b/src/duckdb/src/include/duckdb/main/relation/read_csv_relation.hpp index fc2f98181..cbd60ae91 100644 --- a/src/duckdb/src/include/duckdb/main/relation/read_csv_relation.hpp +++ b/src/duckdb/src/include/duckdb/main/relation/read_csv_relation.hpp @@ -17,13 +17,13 @@ namespace duckdb { class ReadCSVRelation : public TableFunctionRelation { public: - ReadCSVRelation(const shared_ptr &context, const string &csv_file, vector columns, - string alias = string()); - ReadCSVRelation(const shared_ptr &context, const string &csv_file, named_parameter_map_t &&options, - string alias = string()); + ReadCSVRelation(const shared_ptr &context, const vector &csv_files, + named_parameter_map_t &&options, string alias = string()); string alias; - bool auto_detect; + +protected: + void InitializeAlias(const vector &input); public: string GetAlias() override; diff --git a/src/duckdb/src/include/duckdb/main/settings.hpp b/src/duckdb/src/include/duckdb/main/settings.hpp index f0e92267c..65d2b6c4d 100644 --- a/src/duckdb/src/include/duckdb/main/settings.hpp +++ b/src/duckdb/src/include/duckdb/main/settings.hpp @@ -552,22 +552,4 @@ struct FlushAllocatorSetting { static Value GetSetting(ClientContext &context); }; -struct DuckDBApiSetting { - static constexpr const char *Name = "duckdb_api"; - static constexpr const char *Description = "DuckDB API surface"; - static constexpr const LogicalTypeId InputType = LogicalTypeId::VARCHAR; - static void SetGlobal(DatabaseInstance *db, DBConfig &config, const Value ¶meter); - static void ResetGlobal(DatabaseInstance *db, DBConfig &config); - static Value GetSetting(ClientContext &context); -}; - -struct CustomUserAgentSetting { - static constexpr const char *Name = "custom_user_agent"; - static constexpr const char *Description = "Metadata from DuckDB callers"; - static constexpr const LogicalTypeId InputType = LogicalTypeId::VARCHAR; - static void SetGlobal(DatabaseInstance *db, DBConfig &config, const Value ¶meter); - static void ResetGlobal(DatabaseInstance *db, DBConfig &config); - static Value GetSetting(ClientContext &context); -}; - } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/optimizer/join_order/join_node.hpp b/src/duckdb/src/include/duckdb/optimizer/join_order/join_node.hpp index a17721029..f861b2838 100644 --- a/src/duckdb/src/include/duckdb/optimizer/join_order/join_node.hpp +++ b/src/duckdb/src/include/duckdb/optimizer/join_order/join_node.hpp @@ -46,7 +46,7 @@ class JoinNode { private: public: - void PrintJoinNode(); + void Print(); string ToString(); }; diff --git a/src/duckdb/src/include/duckdb/optimizer/join_order/join_relation.hpp b/src/duckdb/src/include/duckdb/optimizer/join_order/join_relation.hpp index aee248e69..7b040c1b5 100644 --- a/src/duckdb/src/include/duckdb/optimizer/join_order/join_relation.hpp +++ b/src/duckdb/src/include/duckdb/optimizer/join_order/join_relation.hpp @@ -49,6 +49,8 @@ class JoinRelationSetManager { JoinRelationSet &Union(JoinRelationSet &left, JoinRelationSet &right); // //! Create the set difference of left \ right (i.e. all elements in left that are not in right) // JoinRelationSet *Difference(JoinRelationSet *left, JoinRelationSet *right); + string ToString() const; + void Print(); private: JoinRelationTreeNode root; diff --git a/src/duckdb/src/include/duckdb/optimizer/rule.hpp b/src/duckdb/src/include/duckdb/optimizer/rule.hpp index 30061a294..a04549fa1 100644 --- a/src/duckdb/src/include/duckdb/optimizer/rule.hpp +++ b/src/duckdb/src/include/duckdb/optimizer/rule.hpp @@ -23,6 +23,8 @@ class Rule { //! The expression rewriter this rule belongs to ExpressionRewriter &rewriter; + //! The root + unique_ptr logical_root; //! The expression matcher of the rule unique_ptr root; diff --git a/src/duckdb/src/include/duckdb/parallel/pipeline_executor.hpp b/src/duckdb/src/include/duckdb/parallel/pipeline_executor.hpp index 631944492..6169425f9 100644 --- a/src/duckdb/src/include/duckdb/parallel/pipeline_executor.hpp +++ b/src/duckdb/src/include/duckdb/parallel/pipeline_executor.hpp @@ -106,10 +106,6 @@ class PipelineExecutor { //! This flag is set when the pipeline gets interrupted by the Sink -> the final_chunk should be re-sink-ed. bool remaining_sink_chunk = false; - //! This flag is set when the pipeline gets interrupted by NextBatch -> NextBatch should be called again and the - //! source_chunk should be sent through the pipeline - bool next_batch_blocked = false; - //! Current operator being flushed idx_t flushing_idx; //! Whether the current flushing_idx should be flushed: this needs to be stored to make flushing code re-entrant @@ -135,9 +131,6 @@ class PipelineExecutor { //! Returns whether or not a new input chunk is needed, or whether or not we are finished OperatorResultType Execute(DataChunk &input, DataChunk &result, idx_t initial_index = 0); - //! Notifies the sink that a new batch has started - SinkNextBatchType NextBatch(DataChunk &source_chunk); - //! Tries to flush all state from intermediate operators. Will return true if all state is flushed, false in the //! case of a blocked sink. bool TryFlushCachingOperators(); @@ -150,7 +143,6 @@ class PipelineExecutor { int debug_blocked_sink_count = 0; int debug_blocked_source_count = 0; int debug_blocked_combine_count = 0; - int debug_blocked_next_batch_count = 0; //! Number of times the Sink/Source will block before actually returning data int debug_blocked_target_count = 1; #endif diff --git a/src/duckdb/src/include/duckdb/parser/expression/lambdaref_expression.hpp b/src/duckdb/src/include/duckdb/parser/expression/lambdaref_expression.hpp new file mode 100644 index 000000000..92408da0a --- /dev/null +++ b/src/duckdb/src/include/duckdb/parser/expression/lambdaref_expression.hpp @@ -0,0 +1,40 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/parser/expression/lambdaref_expression.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/parser/parsed_expression.hpp" + +namespace duckdb { + +//! Represents a reference to a lambda parameter +class LambdaRefExpression : public ParsedExpression { +public: + static constexpr const ExpressionClass TYPE = ExpressionClass::LAMBDA_REF; + +public: + //! Constructs a LambdaRefExpression from a lambda_idx and a column_name. We do not specify a table name, + //! because we use dummy tables to bind lambda parameters + LambdaRefExpression(idx_t lambda_idx, string column_name_p); + + //! The index of the lambda parameter in the lambda_bindings vector + idx_t lambda_idx; + //! The name of the lambda parameter (in a specific Binding in lambda_bindings) + string column_name; + +public: + bool IsScalar() const override; + string GetName() const override; + string ToString() const override; + hash_t Hash() const override; + unique_ptr Copy() const override; + + void Serialize(Serializer &serializer) const override; + static unique_ptr Deserialize(Deserializer &deserializer); +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/expression/list.hpp b/src/duckdb/src/include/duckdb/parser/expression/list.hpp index 7eb8bc8f2..0114a55a2 100644 --- a/src/duckdb/src/include/duckdb/parser/expression/list.hpp +++ b/src/duckdb/src/include/duckdb/parser/expression/list.hpp @@ -4,6 +4,7 @@ #include "duckdb/parser/expression/cast_expression.hpp" #include "duckdb/parser/expression/collate_expression.hpp" #include "duckdb/parser/expression/columnref_expression.hpp" +#include "duckdb/parser/expression/lambdaref_expression.hpp" #include "duckdb/parser/expression/comparison_expression.hpp" #include "duckdb/parser/expression/conjunction_expression.hpp" #include "duckdb/parser/expression/constant_expression.hpp" diff --git a/src/duckdb/src/include/duckdb/parser/expression/window_expression.hpp b/src/duckdb/src/include/duckdb/parser/expression/window_expression.hpp index 14a42304d..9629b0ffe 100644 --- a/src/duckdb/src/include/duckdb/parser/expression/window_expression.hpp +++ b/src/duckdb/src/include/duckdb/parser/expression/window_expression.hpp @@ -25,6 +25,9 @@ enum class WindowBoundary : uint8_t { EXPR_FOLLOWING_RANGE = 8 }; +//! Represents the window exclusion mode +enum class WindowExcludeMode : uint8_t { NO_OTHER = 0, CURRENT_ROW = 1, GROUP = 2, TIES = 3 }; + const char *ToString(WindowBoundary value); //! The WindowExpression represents a window function in the query. They are a special case of aggregates which is why @@ -55,6 +58,8 @@ class WindowExpression : public ParsedExpression { //! The window boundaries WindowBoundary start = WindowBoundary::INVALID; WindowBoundary end = WindowBoundary::INVALID; + //! The EXCLUDE clause + WindowExcludeMode exclude_clause = WindowExcludeMode::NO_OTHER; unique_ptr start_expr; unique_ptr end_expr; @@ -204,6 +209,23 @@ class WindowExpression : public ParsedExpression { result += to; } + if (entry.exclude_clause != WindowExcludeMode::NO_OTHER) { + result += " EXCLUDE "; + } + switch (entry.exclude_clause) { + case WindowExcludeMode::CURRENT_ROW: + result += "CURRENT ROW"; + break; + case WindowExcludeMode::GROUP: + result += "GROUP"; + break; + case WindowExcludeMode::TIES: + result += "TIES"; + break; + default: + break; + } + result += ")"; return result; diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/create_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/create_info.hpp index 70e047286..2d56c112a 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/create_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/create_info.hpp @@ -62,6 +62,7 @@ struct CreateInfo : public ParseInfo { DUCKDB_API void CopyProperties(CreateInfo &other) const; //! Generates an alter statement from the create statement - used for OnCreateConflict::ALTER_ON_CONFLICT DUCKDB_API virtual unique_ptr GetAlterInfo() const; + virtual string ToString() const { throw InternalException("ToString not supported for this type of CreateInfo: '%s'", EnumUtil::ToString(info_type)); diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/create_schema_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/create_schema_info.hpp index 907645873..33d5a8448 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/create_schema_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/create_schema_info.hpp @@ -25,6 +25,29 @@ struct CreateSchemaInfo : public CreateInfo { DUCKDB_API void Serialize(Serializer &serializer) const override; DUCKDB_API static unique_ptr Deserialize(Deserializer &deserializer); + + string ToString() const override { + string ret = ""; + switch (on_conflict) { + case OnCreateConflict::ALTER_ON_CONFLICT: { + ret += "CREATE SCHEMA " + schema + " ON CONFLICT INSERT OR REPLACE;"; + break; + } + case OnCreateConflict::IGNORE_ON_CONFLICT: { + ret += "CREATE SCHEMA " + schema + " IF NOT EXISTS;"; + break; + } + case OnCreateConflict::REPLACE_ON_CONFLICT: { + ret += "CREATE OR REPLACE SCHEMA " + schema + ";"; + break; + } + case OnCreateConflict::ERROR_ON_CONFLICT: { + ret += "CREATE SCHEMA " + schema + ";"; + break; + } + } + return ret; + } }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/create_table_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/create_table_info.hpp index 395f78bac..20a8bd788 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/create_table_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/create_table_info.hpp @@ -30,7 +30,7 @@ struct CreateTableInfo : public CreateInfo { ColumnList columns; //! List of constraints on the table vector> constraints; - //! CREATE TABLE from QUERY + //! CREATE TABLE as QUERY unique_ptr query; public: @@ -38,6 +38,8 @@ struct CreateTableInfo : public CreateInfo { DUCKDB_API void Serialize(Serializer &serializer) const override; DUCKDB_API static unique_ptr Deserialize(Deserializer &deserializer); + + string ToString() const override; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/create_type_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/create_type_info.hpp index c8ab662f8..79f1efaaf 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/create_type_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/create_type_info.hpp @@ -31,6 +31,8 @@ struct CreateTypeInfo : public CreateInfo { DUCKDB_API void Serialize(Serializer &serializer) const override; DUCKDB_API static unique_ptr Deserialize(Deserializer &deserializer); + + string ToString() const override; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/transformer.hpp b/src/duckdb/src/include/duckdb/parser/transformer.hpp index 8371796c0..d16dfb8c4 100644 --- a/src/duckdb/src/include/duckdb/parser/transformer.hpp +++ b/src/duckdb/src/include/duckdb/parser/transformer.hpp @@ -48,7 +48,6 @@ class Transformer { unique_ptr base; unique_ptr column; unique_ptr subquery; - bool has_parameters; }; public: @@ -91,7 +90,7 @@ class Transformer { bool GetParam(const string &name, idx_t &index, PreparedParamType type); void AddPivotEntry(string enum_name, unique_ptr source, unique_ptr column, - unique_ptr subquery, bool has_parameters); + unique_ptr subquery); unique_ptr GenerateCreateEnumStmt(unique_ptr entry); bool HasPivotEntries(); idx_t PivotEntryCount(); diff --git a/src/duckdb/src/include/duckdb/planner/expression/bound_cast_expression.hpp b/src/duckdb/src/include/duckdb/planner/expression/bound_cast_expression.hpp index 595f6267a..2881cedcb 100644 --- a/src/duckdb/src/include/duckdb/planner/expression/bound_cast_expression.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression/bound_cast_expression.hpp @@ -40,6 +40,10 @@ class BoundCastExpression : public Expression { //! Cast an expression to the specified SQL type if required DUCKDB_API static unique_ptr AddCastToType(ClientContext &context, unique_ptr expr, const LogicalType &target_type, bool try_cast = false); + + //! If the expression returns an array, cast it to return a list with the same child type. Otherwise do nothing. + DUCKDB_API static unique_ptr AddArrayCastToList(ClientContext &context, unique_ptr expr); + //! Returns true if a cast is invertible (i.e. CAST(s -> t -> s) = s for all values of s). This is not true for e.g. //! boolean casts, because that can be e.g. -1 -> TRUE -> 1. This is necessary to prevent some optimizer bugs. static bool CastIsInvertible(const LogicalType &source_type, const LogicalType &target_type); diff --git a/src/duckdb/src/include/duckdb/planner/expression/bound_lambda_expression.hpp b/src/duckdb/src/include/duckdb/planner/expression/bound_lambda_expression.hpp index 680650c2a..ba2d8819a 100644 --- a/src/duckdb/src/include/duckdb/planner/expression/bound_lambda_expression.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression/bound_lambda_expression.hpp @@ -21,15 +21,17 @@ class BoundLambdaExpression : public Expression { BoundLambdaExpression(ExpressionType type_p, LogicalType return_type_p, unique_ptr lambda_expr_p, idx_t parameter_count_p); + //! The lambda expression that we'll use in the expression executor during execution unique_ptr lambda_expr; + //! Non-lambda constants, column references, and outer lambda parameters that we need to pass + //! into the execution chunk vector> captures; + //! The number of lhs parameters of the lambda function idx_t parameter_count; public: string ToString() const override; - bool Equals(const BaseExpression &other) const override; - unique_ptr Copy() override; void Serialize(Serializer &serializer) const override; diff --git a/src/duckdb/src/include/duckdb/planner/expression/bound_lambdaref_expression.hpp b/src/duckdb/src/include/duckdb/planner/expression/bound_lambdaref_expression.hpp index 6ede2532a..f7b48ef07 100644 --- a/src/duckdb/src/include/duckdb/planner/expression/bound_lambdaref_expression.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression/bound_lambdaref_expression.hpp @@ -14,21 +14,20 @@ namespace duckdb { //! A BoundLambdaRef expression represents a LambdaRef expression that was bound to an lambda parameter -//! in the lambda bindings vector. When capturing lambdas the BoundLambdaRef becomes a -//! BoundReferenceExpresssion, indexing the corresponding lambda parameter in the lambda bindings vector, +//! in the lambda bindings vector. When capturing lambdas, the BoundLambdaRef becomes a +//! BoundReferenceExpression, indexing the corresponding lambda parameter in the lambda bindings vector, //! which refers to the physical chunk of the lambda parameter during execution. class BoundLambdaRefExpression : public Expression { public: static constexpr const ExpressionClass TYPE = ExpressionClass::BOUND_LAMBDA_REF; public: - BoundLambdaRefExpression(LogicalType type, ColumnBinding binding, idx_t lambda_index, idx_t depth = 0); - BoundLambdaRefExpression(string alias, LogicalType type, ColumnBinding binding, idx_t lambda_index, - idx_t depth = 0); + BoundLambdaRefExpression(LogicalType type, ColumnBinding binding, idx_t lambda_idx, idx_t depth = 0); + BoundLambdaRefExpression(string alias, LogicalType type, ColumnBinding binding, idx_t lambda_idx, idx_t depth = 0); //! Column index set by the binder, used to generate the final BoundExpression ColumnBinding binding; //! The index of the lambda parameter in the lambda bindings vector - idx_t lambda_index; + idx_t lambda_idx; //! The subquery depth (i.e. depth 0 = current query, depth 1 = parent query, depth 2 = parent of parent, etc...). //! This is only non-zero for correlated expressions inside subqueries. idx_t depth; diff --git a/src/duckdb/src/include/duckdb/planner/expression/bound_window_expression.hpp b/src/duckdb/src/include/duckdb/planner/expression/bound_window_expression.hpp index a538344a1..4bc17cac7 100644 --- a/src/duckdb/src/include/duckdb/planner/expression/bound_window_expression.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression/bound_window_expression.hpp @@ -43,6 +43,8 @@ class BoundWindowExpression : public Expression { //! The window boundaries WindowBoundary start = WindowBoundary::INVALID; WindowBoundary end = WindowBoundary::INVALID; + //! The EXCLUDE clause + WindowExcludeMode exclude_clause = WindowExcludeMode::NO_OTHER; unique_ptr start_expr; unique_ptr end_expr; diff --git a/src/duckdb/src/include/duckdb/planner/expression_binder.hpp b/src/duckdb/src/include/duckdb/planner/expression_binder.hpp index 6df014d16..fd5133e52 100644 --- a/src/duckdb/src/include/duckdb/planner/expression_binder.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression_binder.hpp @@ -12,9 +12,12 @@ #include "duckdb/common/stack_checker.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/parser/expression/bound_expression.hpp" +#include "duckdb/parser/expression/lambdaref_expression.hpp" #include "duckdb/parser/parsed_expression.hpp" #include "duckdb/parser/tokens.hpp" #include "duckdb/planner/expression.hpp" +#include "duckdb/planner/expression/bound_lambda_expression.hpp" +#include "duckdb/function/scalar_function.hpp" namespace duckdb { @@ -128,21 +131,25 @@ class ExpressionBinder { BindResult BindExpression(CollateExpression &expr, idx_t depth); BindResult BindExpression(CastExpression &expr, idx_t depth); BindResult BindExpression(ColumnRefExpression &expr, idx_t depth); + BindResult BindExpression(LambdaRefExpression &expr, idx_t depth); BindResult BindExpression(ComparisonExpression &expr, idx_t depth); BindResult BindExpression(ConjunctionExpression &expr, idx_t depth); BindResult BindExpression(ConstantExpression &expr, idx_t depth); BindResult BindExpression(FunctionExpression &expr, idx_t depth, unique_ptr &expr_ptr); - BindResult BindExpression(LambdaExpression &expr, idx_t depth, const bool is_lambda, - const LogicalType &list_child_type); + BindResult BindExpression(LambdaExpression &expr, idx_t depth, const LogicalType &list_child_type, + optional_ptr bind_lambda_function); BindResult BindExpression(OperatorExpression &expr, idx_t depth); BindResult BindExpression(ParameterExpression &expr, idx_t depth); BindResult BindExpression(SubqueryExpression &expr, idx_t depth); BindResult BindPositionalReference(unique_ptr &expr, idx_t depth, bool root_expression); void TransformCapturedLambdaColumn(unique_ptr &original, unique_ptr &replacement, - vector> &captures, LogicalType &list_child_type); - void CaptureLambdaColumns(vector> &captures, LogicalType &list_child_type, - unique_ptr &expr); + BoundLambdaExpression &bound_lambda_expr, + const optional_ptr bind_lambda_function, + const LogicalType &list_child_type); + void CaptureLambdaColumns(BoundLambdaExpression &bound_lambda_expr, unique_ptr &expr, + const optional_ptr bind_lambda_function, + const LogicalType &list_child_type); static unique_ptr GetSQLValueFunction(const string &column_name); diff --git a/src/duckdb/src/include/duckdb/planner/expression_binder/table_function_binder.hpp b/src/duckdb/src/include/duckdb/planner/expression_binder/table_function_binder.hpp index 7f844858d..46dbd1fdd 100644 --- a/src/duckdb/src/include/duckdb/planner/expression_binder/table_function_binder.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression_binder/table_function_binder.hpp @@ -12,12 +12,13 @@ namespace duckdb { -//! The Table function binder can bind standard table function parameters (i.e. non-table-in-out functions) +//! The table function binder can bind standard table function parameters (i.e., non-table-in-out functions) class TableFunctionBinder : public ExpressionBinder { public: TableFunctionBinder(Binder &binder, ClientContext &context); protected: + BindResult BindLambdaReference(LambdaRefExpression &expr, idx_t depth); BindResult BindColumnReference(ColumnRefExpression &expr, idx_t depth, bool root_expression); BindResult BindExpression(unique_ptr &expr, idx_t depth, bool root_expression = false) override; diff --git a/src/duckdb/src/include/duckdb/planner/table_binding.hpp b/src/duckdb/src/include/duckdb/planner/table_binding.hpp index 4d6679c1f..c282deca5 100644 --- a/src/duckdb/src/include/duckdb/planner/table_binding.hpp +++ b/src/duckdb/src/include/duckdb/planner/table_binding.hpp @@ -115,8 +115,8 @@ struct TableBinding : public Binding { ColumnBinding GetColumnBinding(column_t column_index); }; -//! DummyBinding is like the Binding, except the alias and index are set by default. Used for binding lambdas and macro -//! parameters. +//! DummyBinding is like the Binding, except the alias and index are set by default. +//! Used for binding lambdas and macro parameters. struct DummyBinding : public Binding { public: static constexpr const BindingType TYPE = BindingType::DUMMY; @@ -124,16 +124,18 @@ struct DummyBinding : public Binding { static constexpr const char *DUMMY_NAME = "0_macro_parameters"; public: - DummyBinding(vector types_p, vector names_p, string dummy_name_p); + DummyBinding(vector types, vector names, string dummy_name); - //! Arguments + //! Arguments (for macros) vector> *arguments; //! The name of the dummy binding string dummy_name; public: + //! Binding macros BindResult Bind(ColumnRefExpression &colref, idx_t depth) override; - BindResult Bind(ColumnRefExpression &colref, idx_t lambda_index, idx_t depth); + //! Binding lambdas + BindResult Bind(LambdaRefExpression &lambdaref, idx_t depth); //! Given the parameter colref, returns a copy of the argument that was supplied for this parameter unique_ptr ParamToArg(ColumnRefExpression &colref); diff --git a/src/duckdb/src/include/duckdb/storage/statistics/array_stats.hpp b/src/duckdb/src/include/duckdb/storage/statistics/array_stats.hpp new file mode 100644 index 000000000..7a37abc88 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/statistics/array_stats.hpp @@ -0,0 +1,40 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/statistics/array_stats.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/common.hpp" +#include "duckdb/common/exception.hpp" + +namespace duckdb { +class BaseStatistics; +struct SelectionVector; +class Vector; +class Serializer; +class Deserializer; + +struct ArrayStats { + DUCKDB_API static void Construct(BaseStatistics &stats); + DUCKDB_API static BaseStatistics CreateUnknown(LogicalType type); + DUCKDB_API static BaseStatistics CreateEmpty(LogicalType type); + + DUCKDB_API static const BaseStatistics &GetChildStats(const BaseStatistics &stats); + DUCKDB_API static BaseStatistics &GetChildStats(BaseStatistics &stats); + DUCKDB_API static void SetChildStats(BaseStatistics &stats, unique_ptr new_stats); + + DUCKDB_API static void Serialize(const BaseStatistics &stats, Serializer &serializer); + DUCKDB_API static void Deserialize(Deserializer &deserializer, BaseStatistics &base); + + DUCKDB_API static string ToString(const BaseStatistics &stats); + + DUCKDB_API static void Merge(BaseStatistics &stats, const BaseStatistics &other); + DUCKDB_API static void Copy(BaseStatistics &stats, const BaseStatistics &other); + DUCKDB_API static void Verify(const BaseStatistics &stats, Vector &vector, const SelectionVector &sel, idx_t count); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/statistics/base_statistics.hpp b/src/duckdb/src/include/duckdb/storage/statistics/base_statistics.hpp index 790644cc7..c2de957fa 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/base_statistics.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/base_statistics.hpp @@ -33,13 +33,14 @@ enum class StatsInfo : uint8_t { CAN_HAVE_NULL_AND_VALID_VALUES = 4 }; -enum class StatisticsType : uint8_t { NUMERIC_STATS, STRING_STATS, LIST_STATS, STRUCT_STATS, BASE_STATS }; +enum class StatisticsType : uint8_t { NUMERIC_STATS, STRING_STATS, LIST_STATS, STRUCT_STATS, BASE_STATS, ARRAY_STATS }; class BaseStatistics { friend struct NumericStats; friend struct StringStats; friend struct StructStats; friend struct ListStats; + friend struct ArrayStats; public: DUCKDB_API ~BaseStatistics(); diff --git a/src/duckdb/src/include/duckdb/storage/table/array_column_data.hpp b/src/duckdb/src/include/duckdb/storage/table/array_column_data.hpp new file mode 100644 index 000000000..d931067b1 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/table/array_column_data.hpp @@ -0,0 +1,67 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/table/list_column_data.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/table/column_data.hpp" +#include "duckdb/storage/table/validity_column_data.hpp" + +namespace duckdb { + +//! List column data represents a list +class ArrayColumnData : public ColumnData { +public: + ArrayColumnData(BlockManager &block_manager, DataTableInfo &info, idx_t column_index, idx_t start_row, + LogicalType type, optional_ptr parent = nullptr); + + //! The child-column of the list + unique_ptr child_column; + //! The validity column data of the struct + ValidityColumnData validity; + +public: + void SetStart(idx_t new_start) override; + bool CheckZonemap(ColumnScanState &state, TableFilter &filter) override; + + void InitializeScan(ColumnScanState &state) override; + void InitializeScanWithOffset(ColumnScanState &state, idx_t row_idx) override; + + idx_t Scan(TransactionData transaction, idx_t vector_index, ColumnScanState &state, Vector &result) override; + idx_t ScanCommitted(idx_t vector_index, ColumnScanState &state, Vector &result, bool allow_updates) override; + idx_t ScanCount(ColumnScanState &state, Vector &result, idx_t count) override; + + void Skip(ColumnScanState &state, idx_t count = STANDARD_VECTOR_SIZE) override; + + void InitializeAppend(ColumnAppendState &state) override; + void Append(BaseStatistics &stats, ColumnAppendState &state, Vector &vector, idx_t count) override; + void RevertAppend(row_t start_row) override; + idx_t Fetch(ColumnScanState &state, row_t row_id, Vector &result) override; + void FetchRow(TransactionData transaction, ColumnFetchState &state, row_t row_id, Vector &result, + idx_t result_idx) override; + void Update(TransactionData transaction, idx_t column_index, Vector &update_vector, row_t *row_ids, + idx_t update_count) override; + void UpdateColumn(TransactionData transaction, const vector &column_path, Vector &update_vector, + row_t *row_ids, idx_t update_count, idx_t depth) override; + unique_ptr GetUpdateStatistics() override; + + void CommitDropColumn() override; + + unique_ptr CreateCheckpointState(RowGroup &row_group, + PartialBlockManager &partial_block_manager) override; + unique_ptr Checkpoint(RowGroup &row_group, PartialBlockManager &partial_block_manager, + ColumnCheckpointInfo &checkpoint_info) override; + + void DeserializeColumn(Deserializer &source) override; + + void GetColumnSegmentInfo(duckdb::idx_t row_group_index, vector col_path, + vector &result) override; + + void Verify(RowGroup &parent) override; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/main/capi/arrow-c.cpp b/src/duckdb/src/main/capi/arrow-c.cpp index 78dd3e695..9134257fb 100644 --- a/src/duckdb/src/main/capi/arrow-c.cpp +++ b/src/duckdb/src/main/capi/arrow-c.cpp @@ -1,8 +1,8 @@ +#include "duckdb/common/arrow/arrow.hpp" #include "duckdb/common/arrow/arrow_converter.hpp" #include "duckdb/function/table/arrow.hpp" #include "duckdb/main/capi/capi_internal.hpp" #include "duckdb/main/prepared_statement_data.hpp" -#include "duckdb/common/arrow/arrow.hpp" using duckdb::ArrowConverter; using duckdb::ArrowResultWrapper; @@ -28,7 +28,7 @@ duckdb_state duckdb_query_arrow_schema(duckdb_arrow result, duckdb_arrow_schema } auto wrapper = reinterpret_cast(result); ArrowConverter::ToArrowSchema((ArrowSchema *)*out_schema, wrapper->result->types, wrapper->result->names, - wrapper->options); + wrapper->result->client_properties); return DuckDBSuccess; } @@ -83,10 +83,21 @@ duckdb_state duckdb_query_arrow_array(duckdb_arrow result, duckdb_arrow_array *o if (!wrapper->current_chunk || wrapper->current_chunk->size() == 0) { return DuckDBSuccess; } - ArrowConverter::ToArrowArray(*wrapper->current_chunk, reinterpret_cast(*out_array), wrapper->options); + ArrowConverter::ToArrowArray(*wrapper->current_chunk, reinterpret_cast(*out_array), + wrapper->result->client_properties); return DuckDBSuccess; } +void duckdb_result_arrow_array(duckdb_result result, duckdb_data_chunk chunk, duckdb_arrow_array *out_array) { + if (!out_array) { + return; + } + auto dchunk = reinterpret_cast(chunk); + auto &result_data = *(reinterpret_cast(result.internal_data)); + ArrowConverter::ToArrowArray(*dchunk, reinterpret_cast(*out_array), + result_data.result->client_properties); +} + idx_t duckdb_arrow_row_count(duckdb_arrow result) { auto wrapper = reinterpret_cast(result); if (wrapper->result->HasError()) { @@ -136,8 +147,6 @@ duckdb_state duckdb_execute_prepared_arrow(duckdb_prepared_statement prepared_st return DuckDBError; } auto arrow_wrapper = new ArrowResultWrapper(); - arrow_wrapper->options = wrapper->statement->context->GetClientProperties(); - auto result = wrapper->statement->Execute(wrapper->values, false); D_ASSERT(result->type == QueryResultType::MATERIALIZED_RESULT); arrow_wrapper->result = duckdb::unique_ptr_cast(std::move(result)); diff --git a/src/duckdb/src/main/capi/config-c.cpp b/src/duckdb/src/main/capi/config-c.cpp index d607f0bb2..67b4b46b7 100644 --- a/src/duckdb/src/main/capi/config-c.cpp +++ b/src/duckdb/src/main/capi/config-c.cpp @@ -13,7 +13,6 @@ duckdb_state duckdb_create_config(duckdb_config *out_config) { DBConfig *config; try { config = new DBConfig(); - config->SetOptionByName("duckdb_api", "capi"); } catch (...) { // LCOV_EXCL_START return DuckDBError; } // LCOV_EXCL_STOP diff --git a/src/duckdb/src/main/capi/duckdb-c.cpp b/src/duckdb/src/main/capi/duckdb-c.cpp index df522d176..5377df466 100644 --- a/src/duckdb/src/main/capi/duckdb-c.cpp +++ b/src/duckdb/src/main/capi/duckdb-c.cpp @@ -8,15 +8,7 @@ using duckdb::DuckDB; duckdb_state duckdb_open_ext(const char *path, duckdb_database *out, duckdb_config config, char **error) { auto wrapper = new DatabaseData(); try { - DBConfig default_config; - default_config.SetOptionByName("duckdb_api", "capi"); - - DBConfig *db_config = &default_config; - DBConfig *user_config = (DBConfig *)config; - if (user_config) { - db_config = user_config; - } - + auto db_config = (DBConfig *)config; wrapper->database = duckdb::make_uniq(path, db_config); } catch (std::exception &ex) { if (error) { diff --git a/src/duckdb/src/main/config.cpp b/src/duckdb/src/main/config.cpp index ee8f3d2b1..347a8b919 100644 --- a/src/duckdb/src/main/config.cpp +++ b/src/duckdb/src/main/config.cpp @@ -115,8 +115,6 @@ static ConfigurationOption internal_options[] = {DUCKDB_GLOBAL(AccessModeSetting DUCKDB_GLOBAL_ALIAS("wal_autocheckpoint", CheckpointThresholdSetting), DUCKDB_GLOBAL_ALIAS("worker_threads", ThreadsSetting), DUCKDB_GLOBAL(FlushAllocatorSetting), - DUCKDB_GLOBAL(DuckDBApiSetting), - DUCKDB_GLOBAL(CustomUserAgentSetting), FINAL_SETTING}; vector DBConfig::GetOptions() { @@ -420,13 +418,4 @@ OrderByNullType DBConfig::ResolveNullOrder(OrderType order_type, OrderByNullType } } -const std::string DBConfig::UserAgent() const { - auto user_agent = options.duckdb_api; - - if (!options.custom_user_agent.empty()) { - user_agent += " " + options.custom_user_agent; - } - return user_agent; -} - } // namespace duckdb diff --git a/src/duckdb/src/main/connection.cpp b/src/duckdb/src/main/connection.cpp index e225c643c..3d117eb53 100644 --- a/src/duckdb/src/main/connection.cpp +++ b/src/duckdb/src/main/connection.cpp @@ -223,21 +223,29 @@ shared_ptr Connection::ReadCSV(const string &csv_file) { return ReadCSV(csv_file, std::move(options)); } -shared_ptr Connection::ReadCSV(const string &csv_file, named_parameter_map_t &&options) { - return make_shared(context, csv_file, std::move(options)); +shared_ptr Connection::ReadCSV(const vector &csv_input, named_parameter_map_t &&options) { + return make_shared(context, csv_input, std::move(options)); +} + +shared_ptr Connection::ReadCSV(const string &csv_input, named_parameter_map_t &&options) { + vector csv_files = {csv_input}; + return ReadCSV(csv_files, std::move(options)); } shared_ptr Connection::ReadCSV(const string &csv_file, const vector &columns) { // parse columns - vector column_list; + named_parameter_map_t options; + child_list_t column_list; for (auto &column : columns) { auto col_list = Parser::ParseColumnList(column, context->GetParserOptions()); if (col_list.LogicalColumnCount() != 1) { throw ParserException("Expected a single column definition"); } - column_list.push_back(std::move(col_list.GetColumnMutable(LogicalIndex(0)))); + auto &col_def = col_list.GetColumnMutable(LogicalIndex(0)); + column_list.push_back({col_def.GetName(), col_def.GetType().ToString()}); } - return make_shared(context, csv_file, std::move(column_list)); + vector files {csv_file}; + return make_shared(context, files, std::move(options)); } shared_ptr Connection::ReadParquet(const string &parquet_file, bool binary_as_string) { diff --git a/src/duckdb/src/main/database.cpp b/src/duckdb/src/main/database.cpp index a17413061..311038697 100644 --- a/src/duckdb/src/main/database.cpp +++ b/src/duckdb/src/main/database.cpp @@ -31,7 +31,6 @@ DBConfig::DBConfig() { compression_functions = make_uniq(); cast_functions = make_uniq(); error_manager = make_uniq(); - options.duckdb_api = StringUtil::Format("duckdb/%s(%s)", DuckDB::LibraryVersion(), DuckDB::Platform()); } DBConfig::DBConfig(std::unordered_map &config_dict, bool read_only) : DBConfig::DBConfig() { diff --git a/src/duckdb/src/main/relation/read_csv_relation.cpp b/src/duckdb/src/main/relation/read_csv_relation.cpp index fd9ecf552..e8e669261 100644 --- a/src/duckdb/src/main/relation/read_csv_relation.cpp +++ b/src/duckdb/src/main/relation/read_csv_relation.cpp @@ -17,39 +17,32 @@ namespace duckdb { -ReadCSVRelation::ReadCSVRelation(const shared_ptr &context, const string &csv_file, - vector columns_p, string alias_p) - : TableFunctionRelation(context, "read_csv", {Value(csv_file)}, nullptr, false), alias(std::move(alias_p)), - auto_detect(false) { - - if (alias.empty()) { - alias = StringUtil::Split(csv_file, ".")[0]; - } - - columns = std::move(columns_p); +void ReadCSVRelation::InitializeAlias(const vector &input) { + D_ASSERT(!input.empty()); + const auto &csv_file = input[0]; + alias = StringUtil::Split(csv_file, ".")[0]; +} - child_list_t column_names; - for (idx_t i = 0; i < columns.size(); i++) { - column_names.push_back(make_pair(columns[i].Name(), Value(columns[i].Type().ToString()))); +static Value CreateValueFromFileList(const vector &file_list) { + vector files; + for (auto &file : file_list) { + files.push_back(file); } - - AddNamedParameter("columns", Value::STRUCT(std::move(column_names))); + return Value::LIST(std::move(files)); } -ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, const string &csv_file, +ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, const vector &input, named_parameter_map_t &&options, string alias_p) - : TableFunctionRelation(context, "read_csv_auto", {Value(csv_file)}, nullptr, false), alias(std::move(alias_p)), - auto_detect(true) { + : TableFunctionRelation(context, "read_csv_auto", {CreateValueFromFileList(input)}, nullptr, false), + alias(std::move(alias_p)) { - if (alias.empty()) { - alias = StringUtil::Split(csv_file, ".")[0]; - } + InitializeAlias(input); - auto files = MultiFileReader::GetFileList(*context, csv_file, "CSV"); + auto file_list = CreateValueFromFileList(input); + auto files = MultiFileReader::GetFileList(*context, file_list, "CSV"); D_ASSERT(!files.empty()); auto &file_name = files[0]; - options["auto_detect"] = Value::BOOLEAN(true); CSVReaderOptions csv_options; csv_options.file_path = file_name; vector empty; @@ -57,8 +50,8 @@ ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, vector unused_types; vector unused_names; csv_options.FromNamedParameters(options, *context, unused_types, unused_names); - // Run the auto-detect, populating the options with the detected settings + // Run the auto-detect, populating the options with the detected settings auto bm_file_handle = BaseCSVReader::OpenCSV(*context, csv_options); auto buffer_manager = make_shared(*context, std::move(bm_file_handle), csv_options); CSVStateMachineCache state_machine_cache; @@ -70,12 +63,24 @@ ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, columns.emplace_back(names[i], types[i]); } + // After sniffing we can consider these set, so they are exported as named parameters + csv_options.has_delimiter = true; + csv_options.has_quote = true; + csv_options.has_escape = true; + //! Capture the options potentially set/altered by the auto detection phase csv_options.ToNamedParameters(options); // No need to auto-detect again options["auto_detect"] = Value::BOOLEAN(false); SetNamedParameters(std::move(options)); + + child_list_t column_names; + for (idx_t i = 0; i < columns.size(); i++) { + column_names.push_back(make_pair(columns[i].Name(), Value(columns[i].Type().ToString()))); + } + + AddNamedParameter("columns", Value::STRUCT(std::move(column_names))); } string ReadCSVRelation::GetAlias() { diff --git a/src/duckdb/src/main/settings/settings.cpp b/src/duckdb/src/main/settings/settings.cpp index e75c8f500..1fe56fed5 100644 --- a/src/duckdb/src/main/settings/settings.cpp +++ b/src/duckdb/src/main/settings/settings.cpp @@ -1203,53 +1203,4 @@ Value FlushAllocatorSetting::GetSetting(ClientContext &context) { return Value(StringUtil::BytesToHumanReadableString(config.options.allocator_flush_threshold)); } -//===--------------------------------------------------------------------===// -// DuckDBApi Setting -//===--------------------------------------------------------------------===// - -void DuckDBApiSetting::SetGlobal(DatabaseInstance *db, DBConfig &config, const Value &input) { - auto new_value = input.GetValue(); - if (db) { - throw InvalidInputException("Cannot change duckdb_api setting while database is running"); - } - config.options.duckdb_api += " " + new_value; -} - -void DuckDBApiSetting::ResetGlobal(DatabaseInstance *db, DBConfig &config) { - if (db) { - throw InvalidInputException("Cannot change duckdb_api setting while database is running"); - } - config.options.duckdb_api = DBConfig().options.duckdb_api; -} - -Value DuckDBApiSetting::GetSetting(ClientContext &context) { - auto &config = DBConfig::GetConfig(context); - return Value(config.options.duckdb_api); -} - -//===--------------------------------------------------------------------===// -// CustomUserAgent Setting -//===--------------------------------------------------------------------===// - -void CustomUserAgentSetting::SetGlobal(DatabaseInstance *db, DBConfig &config, const Value &input) { - auto new_value = input.GetValue(); - if (db) { - throw InvalidInputException("Cannot change custom_user_agent setting while database is running"); - } - config.options.custom_user_agent = - config.options.custom_user_agent.empty() ? new_value : config.options.custom_user_agent + " " + new_value; -} - -void CustomUserAgentSetting::ResetGlobal(DatabaseInstance *db, DBConfig &config) { - if (db) { - throw InvalidInputException("Cannot change custom_user_agent setting while database is running"); - } - config.options.custom_user_agent = DBConfig().options.custom_user_agent; -} - -Value CustomUserAgentSetting::GetSetting(ClientContext &context) { - auto &config = DBConfig::GetConfig(context); - return Value(config.options.custom_user_agent); -} - } // namespace duckdb diff --git a/src/duckdb/src/optimizer/expression_rewriter.cpp b/src/duckdb/src/optimizer/expression_rewriter.cpp index fea861e9d..2e3a19f20 100644 --- a/src/duckdb/src/optimizer/expression_rewriter.cpp +++ b/src/duckdb/src/optimizer/expression_rewriter.cpp @@ -59,8 +59,16 @@ void ExpressionRewriter::VisitOperator(LogicalOperator &op) { to_apply_rules.clear(); for (auto &rule : rules) { + if (rule->logical_root && !rule->logical_root->Match(op.type)) { + // this rule does not apply to this type of LogicalOperator + continue; + } to_apply_rules.push_back(*rule); } + if (to_apply_rules.empty()) { + // no rules to apply on this node + return; + } VisitOperatorExpressions(op); diff --git a/src/duckdb/src/optimizer/join_order/join_node.cpp b/src/duckdb/src/optimizer/join_order/join_node.cpp index 1786bd499..ef8baa084 100644 --- a/src/duckdb/src/optimizer/join_order/join_node.cpp +++ b/src/duckdb/src/optimizer/join_order/join_node.cpp @@ -33,4 +33,8 @@ string JoinNode::ToString() { return result; } +void JoinNode::Print() { + Printer::Print(ToString()); +} + } // namespace duckdb diff --git a/src/duckdb/src/optimizer/join_order/join_relation_set.cpp b/src/duckdb/src/optimizer/join_order/join_relation_set.cpp index 576d5eabe..d08ac4c89 100644 --- a/src/duckdb/src/optimizer/join_order/join_relation_set.cpp +++ b/src/duckdb/src/optimizer/join_order/join_relation_set.cpp @@ -1,4 +1,5 @@ #include "duckdb/optimizer/join_order/join_relation.hpp" +#include "duckdb/common/printer.hpp" #include "duckdb/common/string_util.hpp" #include "duckdb/common/to_string.hpp" @@ -139,4 +140,23 @@ JoinRelationSet &JoinRelationSetManager::Union(JoinRelationSet &left, JoinRelati // return GetJoinRelation(std::move(relations), count); // } +static string JoinRelationTreeNodeToString(const JoinRelationTreeNode *node) { + string result = ""; + if (node->relation) { + result += node->relation.get()->ToString() + "\n"; + } + for (auto &child : node->children) { + result += JoinRelationTreeNodeToString(child.second.get()); + } + return result; +} + +string JoinRelationSetManager::ToString() const { + return JoinRelationTreeNodeToString(&root); +} + +void JoinRelationSetManager::Print() { + Printer::Print(ToString()); +} + } // namespace duckdb diff --git a/src/duckdb/src/optimizer/statistics/operator/propagate_join.cpp b/src/duckdb/src/optimizer/statistics/operator/propagate_join.cpp index 810d0054e..d1d7bd5b7 100644 --- a/src/duckdb/src/optimizer/statistics/operator/propagate_join.cpp +++ b/src/duckdb/src/optimizer/statistics/operator/propagate_join.cpp @@ -108,6 +108,10 @@ void StatisticsPropagator::PropagateStatistics(LogicalComparisonJoin &join, uniq *node_ptr = std::move(cross_product); return; } + case JoinType::ANTI: + // anti join on true: empty result + ReplaceWithEmptyResult(*node_ptr); + return; default: // we don't handle mark/single join here yet break; diff --git a/src/duckdb/src/parallel/pipeline_executor.cpp b/src/duckdb/src/parallel/pipeline_executor.cpp index c495d4080..80fc57ca0 100644 --- a/src/duckdb/src/parallel/pipeline_executor.cpp +++ b/src/duckdb/src/parallel/pipeline_executor.cpp @@ -106,58 +106,6 @@ bool PipelineExecutor::TryFlushCachingOperators() { return true; } -SinkNextBatchType PipelineExecutor::NextBatch(duckdb::DataChunk &source_chunk) { - D_ASSERT(requires_batch_index); - idx_t next_batch_index; - if (source_chunk.size() == 0) { - next_batch_index = NumericLimits::Maximum(); - } else { - next_batch_index = - pipeline.source->GetBatchIndex(context, source_chunk, *pipeline.source_state, *local_source_state); - // we start with the base_batch_index as a valid starting value. Make sure that next batch is called below - next_batch_index += pipeline.base_batch_index + 1; - } - auto &partition_info = local_sink_state->partition_info; - if (next_batch_index == partition_info.batch_index.GetIndex()) { - // no changes, return - return SinkNextBatchType::READY; - } - // batch index has changed - update it - if (partition_info.batch_index.GetIndex() > next_batch_index) { - throw InternalException( - "Pipeline batch index - gotten lower batch index %llu (down from previous batch index of %llu)", - next_batch_index, partition_info.batch_index.GetIndex()); - } -#ifdef DUCKDB_DEBUG_ASYNC_SINK_SOURCE - if (debug_blocked_next_batch_count < debug_blocked_target_count) { - debug_blocked_next_batch_count++; - - auto &callback_state = interrupt_state; - std::thread rewake_thread([callback_state] { - std::this_thread::sleep_for(std::chrono::milliseconds(1)); - callback_state.Callback(); - }); - rewake_thread.detach(); - - return SinkNextBatchType::BLOCKED; - } -#endif - auto current_batch = partition_info.batch_index.GetIndex(); - partition_info.batch_index = next_batch_index; - OperatorSinkNextBatchInput next_batch_input {*pipeline.sink->sink_state, *local_sink_state, interrupt_state}; - // call NextBatch before updating min_batch_index to provide the opportunity to flush the previous batch - auto next_batch_result = pipeline.sink->NextBatch(context, next_batch_input); - - if (next_batch_result == SinkNextBatchType::BLOCKED) { - partition_info.batch_index = current_batch; // set batch_index back to what it was before - return SinkNextBatchType::BLOCKED; - } - - partition_info.min_batch_index = pipeline.UpdateBatchIndex(current_batch, next_batch_index); - - return SinkNextBatchType::READY; -} - PipelineExecuteResult PipelineExecutor::Execute(idx_t max_chunks) { D_ASSERT(pipeline.sink); auto &source_chunk = pipeline.operators.empty() ? final_chunk : *intermediate_chunks[0]; @@ -167,8 +115,7 @@ PipelineExecuteResult PipelineExecutor::Execute(idx_t max_chunks) { } OperatorResultType result; - if (exhausted_source && done_flushing && !remaining_sink_chunk && !next_batch_blocked && - in_process_operators.empty()) { + if (exhausted_source && done_flushing && !remaining_sink_chunk && in_process_operators.empty()) { break; } else if (remaining_sink_chunk) { // The pipeline was interrupted by the Sink. We should retry sinking the final chunk. @@ -180,7 +127,7 @@ PipelineExecuteResult PipelineExecutor::Execute(idx_t max_chunks) { // the result for the pipeline D_ASSERT(source_chunk.size() > 0); result = ExecutePushInternal(source_chunk); - } else if (exhausted_source && !next_batch_blocked && !done_flushing) { + } else if (exhausted_source && !done_flushing) { // The source was exhausted, try flushing all operators auto flush_completed = TryFlushCachingOperators(); if (flush_completed) { @@ -189,33 +136,21 @@ PipelineExecuteResult PipelineExecutor::Execute(idx_t max_chunks) { } else { return PipelineExecuteResult::INTERRUPTED; } - } else if (!exhausted_source || next_batch_blocked) { - SourceResultType source_result; - if (!next_batch_blocked) { - // "Regular" path: fetch a chunk from the source and push it through the pipeline - source_chunk.Reset(); - source_result = FetchFromSource(source_chunk); - if (source_result == SourceResultType::BLOCKED) { - return PipelineExecuteResult::INTERRUPTED; - } - if (source_result == SourceResultType::FINISHED) { - exhausted_source = true; - } - } + } else if (!exhausted_source) { + // "Regular" path: fetch a chunk from the source and push it through the pipeline + source_chunk.Reset(); + SourceResultType source_result = FetchFromSource(source_chunk); - if (requires_batch_index) { - auto next_batch_result = NextBatch(source_chunk); - next_batch_blocked = next_batch_result == SinkNextBatchType::BLOCKED; - if (next_batch_blocked) { - return PipelineExecuteResult::INTERRUPTED; - } + if (source_result == SourceResultType::BLOCKED) { + return PipelineExecuteResult::INTERRUPTED; } - if (exhausted_source && source_chunk.size() == 0) { - // To ensure that we're not early-terminating the pipeline - continue; + if (source_result == SourceResultType::FINISHED) { + exhausted_source = true; + if (source_chunk.size() == 0) { + continue; + } } - result = ExecutePushInternal(source_chunk); } else { throw InternalException("Unexpected state reached in pipeline executor"); @@ -350,9 +285,8 @@ void PipelineExecutor::ExecutePull(DataChunk &result) { auto &executor = pipeline.executor; try { D_ASSERT(!pipeline.sink); - D_ASSERT(!requires_batch_index); auto &source_chunk = pipeline.operators.empty() ? result : *intermediate_chunks[0]; - while (result.size() == 0 && (!exhausted_source || !in_process_operators.empty())) { + while (result.size() == 0 && !exhausted_source) { if (in_process_operators.empty()) { source_chunk.Reset(); @@ -362,7 +296,6 @@ void PipelineExecutor::ExecutePull(DataChunk &result) { // Repeatedly try to fetch from the source until it doesn't block. Note that it may block multiple times while (true) { - D_ASSERT(!exhausted_source); source_result = FetchFromSource(source_chunk); // No interrupt happened, all good. @@ -556,6 +489,32 @@ SourceResultType PipelineExecutor::FetchFromSource(DataChunk &result) { // Ensures Sinks only return empty results when Blocking or Finished D_ASSERT(res != SourceResultType::BLOCKED || result.size() == 0); + if (requires_batch_index && res != SourceResultType::BLOCKED) { + idx_t next_batch_index; + if (result.size() == 0) { + next_batch_index = NumericLimits::Maximum(); + } else { + next_batch_index = + pipeline.source->GetBatchIndex(context, result, *pipeline.source_state, *local_source_state); + // we start with the base_batch_index as a valid starting value. Make sure that next batch is called below + next_batch_index += pipeline.base_batch_index + 1; + } + auto &partition_info = local_sink_state->partition_info; + if (next_batch_index != partition_info.batch_index.GetIndex()) { + // batch index has changed - update it + if (partition_info.batch_index.GetIndex() > next_batch_index) { + throw InternalException( + "Pipeline batch index - gotten lower batch index %llu (down from previous batch index of %llu)", + next_batch_index, partition_info.batch_index.GetIndex()); + } + auto current_batch = partition_info.batch_index.GetIndex(); + partition_info.batch_index = next_batch_index; + // call NextBatch before updating min_batch_index to provide the opportunity to flush the previous batch + pipeline.sink->NextBatch(context, *pipeline.sink->sink_state, *local_sink_state); + partition_info.min_batch_index = pipeline.UpdateBatchIndex(current_batch, next_batch_index); + } + } + EndOperator(*pipeline.source, &result); return res; diff --git a/src/duckdb/src/parser/expression/lambdaref_expression.cpp b/src/duckdb/src/parser/expression/lambdaref_expression.cpp new file mode 100644 index 000000000..461e4a348 --- /dev/null +++ b/src/duckdb/src/parser/expression/lambdaref_expression.cpp @@ -0,0 +1,36 @@ +#include "duckdb/parser/expression/lambdaref_expression.hpp" + +#include "duckdb/common/types/hash.hpp" + +namespace duckdb { + +LambdaRefExpression::LambdaRefExpression(idx_t lambda_idx, string column_name_p) + : ParsedExpression(ExpressionType::LAMBDA_REF, ExpressionClass::LAMBDA_REF), lambda_idx(lambda_idx), + column_name(std::move(column_name_p)) { + alias = column_name; +} + +bool LambdaRefExpression::IsScalar() const { + throw InternalException("lambda reference expressions are transient, IsScalar should never be called"); +} + +string LambdaRefExpression::GetName() const { + return column_name; +} + +string LambdaRefExpression::ToString() const { + throw InternalException("lambda reference expressions are transient, ToString should never be called"); +} + +hash_t LambdaRefExpression::Hash() const { + hash_t result = ParsedExpression::Hash(); + result = CombineHash(result, lambda_idx); + result = CombineHash(result, StringUtil::CIHash(column_name)); + return result; +} + +unique_ptr LambdaRefExpression::Copy() const { + throw InternalException("lambda reference expressions are transient, Copy should never be called"); +} + +} // namespace duckdb diff --git a/src/duckdb/src/parser/expression/window_expression.cpp b/src/duckdb/src/parser/expression/window_expression.cpp index c36228e2f..ca1990d30 100644 --- a/src/duckdb/src/parser/expression/window_expression.cpp +++ b/src/duckdb/src/parser/expression/window_expression.cpp @@ -76,6 +76,9 @@ bool WindowExpression::Equal(const WindowExpression &a, const WindowExpression & if (a.start != b.start || a.end != b.end) { return false; } + if (a.exclude_clause != b.exclude_clause) { + return false; + } // check if the framing expressions are equivalentbind_ if (!ParsedExpression::Equals(a.start_expr, b.start_expr) || !ParsedExpression::Equals(a.end_expr, b.end_expr) || !ParsedExpression::Equals(a.offset_expr, b.offset_expr) || @@ -127,6 +130,7 @@ unique_ptr WindowExpression::Copy() const { new_window->start = start; new_window->end = end; + new_window->exclude_clause = exclude_clause; new_window->start_expr = start_expr ? start_expr->Copy() : nullptr; new_window->end_expr = end_expr ? end_expr->Copy() : nullptr; new_window->offset_expr = offset_expr ? offset_expr->Copy() : nullptr; diff --git a/src/duckdb/src/parser/parsed_data/create_table_info.cpp b/src/duckdb/src/parser/parsed_data/create_table_info.cpp index e97e277df..69d1f8dab 100644 --- a/src/duckdb/src/parser/parsed_data/create_table_info.cpp +++ b/src/duckdb/src/parser/parsed_data/create_table_info.cpp @@ -29,4 +29,21 @@ unique_ptr CreateTableInfo::Copy() const { return std::move(result); } +string CreateTableInfo::ToString() const { + string ret = ""; + + string table_name = KeywordHelper::WriteOptionallyQuoted(table); + if (schema != DEFAULT_SCHEMA) { + table_name = KeywordHelper::WriteOptionallyQuoted(schema) + "." + table_name; + } + + ret += "CREATE TABLE " + table_name; + if (query != nullptr) { + ret += " AS " + query->ToString(); + } else { + ret += TableCatalogEntry::ColumnsToSQL(columns, constraints) + ";"; + } + return ret; +} + } // namespace duckdb diff --git a/src/duckdb/src/parser/parsed_data/create_type_info.cpp b/src/duckdb/src/parser/parsed_data/create_type_info.cpp index 3d4bbb7a8..176038918 100644 --- a/src/duckdb/src/parser/parsed_data/create_type_info.cpp +++ b/src/duckdb/src/parser/parsed_data/create_type_info.cpp @@ -22,4 +22,23 @@ unique_ptr CreateTypeInfo::Copy() const { return std::move(result); } +string CreateTypeInfo::ToString() const { + string result = ""; + D_ASSERT(type.id() == LogicalTypeId::ENUM); + auto &values_insert_order = EnumType::GetValuesInsertOrder(type); + idx_t size = EnumType::GetSize(type); + result += "CREATE TYPE "; + result += KeywordHelper::WriteOptionallyQuoted(name); + result += " AS ENUM ( "; + + for (idx_t i = 0; i < size; i++) { + result += "'" + values_insert_order.GetValue(i).ToString() + "'"; + if (i != size - 1) { + result += ", "; + } + } + result += " );"; + return result; +} + } // namespace duckdb diff --git a/src/duckdb/src/parser/parsed_expression_iterator.cpp b/src/duckdb/src/parser/parsed_expression_iterator.cpp index be997f03d..77d60316a 100644 --- a/src/duckdb/src/parser/parsed_expression_iterator.cpp +++ b/src/duckdb/src/parser/parsed_expression_iterator.cpp @@ -141,6 +141,7 @@ void ParsedExpressionIterator::EnumerateChildren( } case ExpressionClass::BOUND_EXPRESSION: case ExpressionClass::COLUMN_REF: + case ExpressionClass::LAMBDA_REF: case ExpressionClass::CONSTANT: case ExpressionClass::DEFAULT: case ExpressionClass::PARAMETER: diff --git a/src/duckdb/src/parser/transform/expression/transform_function.cpp b/src/duckdb/src/parser/transform/expression/transform_function.cpp index bca232b42..c51bd35e9 100644 --- a/src/duckdb/src/parser/transform/expression/transform_function.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_function.cpp @@ -41,12 +41,15 @@ void Transformer::TransformWindowFrame(duckdb_libpgquery::PGWindowDef &window_sp "Window frames starting with unbounded following or ending in unbounded preceding make no sense"); } + if (window_spec.frameOptions & FRAMEOPTION_GROUPS) { + throw ParserException("GROUPS mode for window functions is not implemented yet"); + } const bool rangeMode = (window_spec.frameOptions & FRAMEOPTION_RANGE) != 0; if (window_spec.frameOptions & FRAMEOPTION_START_UNBOUNDED_PRECEDING) { expr.start = WindowBoundary::UNBOUNDED_PRECEDING; - } else if (window_spec.frameOptions & FRAMEOPTION_START_VALUE_PRECEDING) { + } else if (window_spec.frameOptions & FRAMEOPTION_START_OFFSET_PRECEDING) { expr.start = rangeMode ? WindowBoundary::EXPR_PRECEDING_RANGE : WindowBoundary::EXPR_PRECEDING_ROWS; - } else if (window_spec.frameOptions & FRAMEOPTION_START_VALUE_FOLLOWING) { + } else if (window_spec.frameOptions & FRAMEOPTION_START_OFFSET_FOLLOWING) { expr.start = rangeMode ? WindowBoundary::EXPR_FOLLOWING_RANGE : WindowBoundary::EXPR_FOLLOWING_ROWS; } else if (window_spec.frameOptions & FRAMEOPTION_START_CURRENT_ROW) { expr.start = rangeMode ? WindowBoundary::CURRENT_ROW_RANGE : WindowBoundary::CURRENT_ROW_ROWS; @@ -54,21 +57,31 @@ void Transformer::TransformWindowFrame(duckdb_libpgquery::PGWindowDef &window_sp if (window_spec.frameOptions & FRAMEOPTION_END_UNBOUNDED_FOLLOWING) { expr.end = WindowBoundary::UNBOUNDED_FOLLOWING; - } else if (window_spec.frameOptions & FRAMEOPTION_END_VALUE_PRECEDING) { + } else if (window_spec.frameOptions & FRAMEOPTION_END_OFFSET_PRECEDING) { expr.end = rangeMode ? WindowBoundary::EXPR_PRECEDING_RANGE : WindowBoundary::EXPR_PRECEDING_ROWS; - } else if (window_spec.frameOptions & FRAMEOPTION_END_VALUE_FOLLOWING) { + } else if (window_spec.frameOptions & FRAMEOPTION_END_OFFSET_FOLLOWING) { expr.end = rangeMode ? WindowBoundary::EXPR_FOLLOWING_RANGE : WindowBoundary::EXPR_FOLLOWING_ROWS; } else if (window_spec.frameOptions & FRAMEOPTION_END_CURRENT_ROW) { expr.end = rangeMode ? WindowBoundary::CURRENT_ROW_RANGE : WindowBoundary::CURRENT_ROW_ROWS; } D_ASSERT(expr.start != WindowBoundary::INVALID && expr.end != WindowBoundary::INVALID); - if (((window_spec.frameOptions & (FRAMEOPTION_START_VALUE_PRECEDING | FRAMEOPTION_START_VALUE_FOLLOWING)) && + if (((window_spec.frameOptions & (FRAMEOPTION_START_OFFSET_PRECEDING | FRAMEOPTION_START_OFFSET_FOLLOWING)) && !expr.start_expr) || - ((window_spec.frameOptions & (FRAMEOPTION_END_VALUE_PRECEDING | FRAMEOPTION_END_VALUE_FOLLOWING)) && + ((window_spec.frameOptions & (FRAMEOPTION_END_OFFSET_PRECEDING | FRAMEOPTION_END_OFFSET_FOLLOWING)) && !expr.end_expr)) { throw InternalException("Failed to transform window boundary expression"); } + + if (window_spec.frameOptions & FRAMEOPTION_EXCLUDE_CURRENT_ROW) { + expr.exclude_clause = WindowExcludeMode::CURRENT_ROW; + } else if (window_spec.frameOptions & FRAMEOPTION_EXCLUDE_GROUP) { + expr.exclude_clause = WindowExcludeMode::GROUP; + } else if (window_spec.frameOptions & FRAMEOPTION_EXCLUDE_TIES) { + expr.exclude_clause = WindowExcludeMode::TIES; + } else { + expr.exclude_clause = WindowExcludeMode::NO_OTHER; + } } bool Transformer::ExpressionIsEmptyStar(ParsedExpression &expr) { diff --git a/src/duckdb/src/parser/transform/helpers/transform_typename.cpp b/src/duckdb/src/parser/transform/helpers/transform_typename.cpp index 2527b55f1..11ea3331e 100644 --- a/src/duckdb/src/parser/transform/helpers/transform_typename.cpp +++ b/src/duckdb/src/parser/transform/helpers/transform_typename.cpp @@ -224,8 +224,23 @@ LogicalType Transformer::TransformTypeName(duckdb_libpgquery::PGTypeName &type_n // array bounds: turn the type into a list idx_t extra_stack = 0; for (auto cell = type_name.arrayBounds->head; cell != nullptr; cell = cell->next) { - result_type = LogicalType::LIST(result_type); StackCheck(extra_stack++); + auto val = PGPointerCast(cell->data.ptr_value); + if (val->type != duckdb_libpgquery::T_PGInteger) { + throw ParserException("Expected integer value as array bound"); + } + auto array_size = val->val.ival; + if (array_size < 0) { + // -1 if bounds are empty + result_type = LogicalType::LIST(result_type); + } else if (array_size == 0) { + // Empty arrays are not supported + throw ParserException("Arrays must have a size of at least 1"); + } else if (array_size > static_cast(ArrayType::MAX_ARRAY_SIZE)) { + throw ParserException("Arrays must have a size of at most %d", ArrayType::MAX_ARRAY_SIZE); + } else { + result_type = LogicalType::ARRAY(result_type, array_size); + } } } return result_type; diff --git a/src/duckdb/src/parser/transform/statement/transform_pivot_stmt.cpp b/src/duckdb/src/parser/transform/statement/transform_pivot_stmt.cpp index a554b2a6a..849af1a34 100644 --- a/src/duckdb/src/parser/transform/statement/transform_pivot_stmt.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_pivot_stmt.cpp @@ -19,10 +19,9 @@ namespace duckdb { void Transformer::AddPivotEntry(string enum_name, unique_ptr base, unique_ptr column, - unique_ptr subquery, bool has_parameters) { + unique_ptr subquery) { if (parent) { - parent->AddPivotEntry(std::move(enum_name), std::move(base), std::move(column), std::move(subquery), - has_parameters); + parent->AddPivotEntry(std::move(enum_name), std::move(base), std::move(column), std::move(subquery)); return; } auto result = make_uniq(); @@ -30,7 +29,6 @@ void Transformer::AddPivotEntry(string enum_name, unique_ptr base, u result->base = std::move(base); result->column = std::move(column); result->subquery = std::move(subquery); - result->has_parameters = has_parameters; pivot_entries.push_back(std::move(result)); } @@ -115,13 +113,6 @@ unique_ptr Transformer::GenerateCreateEnumStmt(unique_ptr Transformer::CreatePivotStatement(unique_ptr statement) { auto result = make_uniq(); for (auto &pivot : pivot_entries) { - if (pivot->has_parameters) { - throw ParserException( - "PIVOT statements with pivot elements extracted from the data cannot have parameters in their source.\n" - "In order to use parameters the PIVOT values must be manually specified, e.g.:\n" - "PIVOT ... ON %s IN (val1, val2, ...)", - pivot->column->ToString()); - } result->statements.push_back(GenerateCreateEnumStmt(std::move(pivot))); } result->statements.push_back(std::move(statement)); @@ -134,10 +125,7 @@ unique_ptr Transformer::CreatePivotStatement(unique_ptr Transformer::TransformPivotStatement(duckdb_libpgquery::PGSelectStmt &select) { auto pivot = select.pivot; - auto current_param_count = ParamCount(); auto source = TransformTableRefNode(*pivot->source); - auto next_param_count = ParamCount(); - bool has_parameters = next_param_count > current_param_count; auto select_node = make_uniq(); vector> materialized_ctes; @@ -183,8 +171,7 @@ unique_ptr Transformer::TransformPivotStatement(duckdb_libpgquery::PG auto new_select = make_uniq(); ExtractCTEsRecursive(new_select->cte_map); new_select->from_table = source->Copy(); - AddPivotEntry(enum_name, std::move(new_select), col.pivot_expressions[0]->Copy(), std::move(col.subquery), - has_parameters); + AddPivotEntry(enum_name, std::move(new_select), col.pivot_expressions[0]->Copy(), std::move(col.subquery)); col.pivot_enum = enum_name; } diff --git a/src/duckdb/src/planner/binder/expression/bind_columnref_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_columnref_expression.cpp index 47f949624..e59114dba 100644 --- a/src/duckdb/src/planner/binder/expression/bind_columnref_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_columnref_expression.cpp @@ -10,7 +10,6 @@ #include "duckdb/parser/parsed_expression_iterator.hpp" #include "duckdb/planner/binder.hpp" #include "duckdb/planner/expression/bound_columnref_expression.hpp" -#include "duckdb/planner/expression/bound_lambdaref_expression.hpp" #include "duckdb/planner/expression/bound_constant_expression.hpp" #include "duckdb/planner/expression_binder.hpp" #include "duckdb/planner/expression_binder/where_binder.hpp" @@ -97,7 +96,7 @@ unique_ptr ExpressionBinder::QualifyColumnName(const string &c } D_ASSERT(!(*lambda_bindings)[i].alias.empty()); - return make_uniq(column_name, (*lambda_bindings)[i].alias); + return make_uniq(i, column_name); } } } @@ -164,20 +163,6 @@ void ExpressionBinder::QualifyColumnNames(Binder &binder, unique_ptr ExpressionBinder::CreateStructExtract(unique_ptr base, string field_name) { - // we need to transform the struct extract if it is inside a lambda expression - // because we cannot bind to an existing table, so we remove the dummy table also - if (lambda_bindings && base->type == ExpressionType::COLUMN_REF) { - auto &lambda_column_ref = base->Cast(); - D_ASSERT(!lambda_column_ref.column_names.empty()); - - if (lambda_column_ref.column_names[0].find(DummyBinding::DUMMY_NAME) != string::npos) { - D_ASSERT(lambda_column_ref.column_names.size() == 2); - auto lambda_param_name = lambda_column_ref.column_names.back(); - lambda_column_ref.column_names.clear(); - lambda_column_ref.column_names.push_back(lambda_param_name); - } - } - vector> children; children.push_back(std::move(base)); children.push_back(make_uniq_base(Value(std::move(field_name)))); @@ -326,6 +311,11 @@ unique_ptr ExpressionBinder::QualifyColumnName(ColumnRefExpres } } +BindResult ExpressionBinder::BindExpression(LambdaRefExpression &lambdaref, idx_t depth) { + D_ASSERT(lambda_bindings && lambdaref.lambda_idx < lambda_bindings->size()); + return (*lambda_bindings)[lambdaref.lambda_idx].Bind(lambdaref, depth); +} + BindResult ExpressionBinder::BindExpression(ColumnRefExpression &colref_p, idx_t depth) { if (binder.GetBindingMode() == BindingMode::EXTRACT_NAMES) { return BindResult(make_uniq(Value(LogicalType::SQLNULL))); @@ -354,27 +344,13 @@ BindResult ExpressionBinder::BindExpression(ColumnRefExpression &colref_p, idx_t // individual column reference // resolve to either a base table or a subquery expression // if it was a macro parameter, let macro_binding bind it to the argument - // if it was a lambda parameter, let lambda_bindings bind it to the argument BindResult result; - auto found_lambda_binding = false; - if (lambda_bindings) { - for (idx_t i = 0; i < lambda_bindings->size(); i++) { - if (table_name == (*lambda_bindings)[i].alias) { - result = (*lambda_bindings)[i].Bind(colref, i, depth); - found_lambda_binding = true; - break; - } - } - } - - if (!found_lambda_binding) { - if (binder.macro_binding && table_name == binder.macro_binding->alias) { - result = binder.macro_binding->Bind(colref, depth); - } else { - result = binder.bind_context.BindColumn(colref, depth); - } + if (binder.macro_binding && table_name == binder.macro_binding->alias) { + result = binder.macro_binding->Bind(colref, depth); + } else { + result = binder.bind_context.BindColumn(colref, depth); } if (!result.HasError()) { diff --git a/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp index c2a6179ed..08003c2e9 100644 --- a/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp @@ -82,16 +82,25 @@ BindResult ExpressionBinder::BindExpression(FunctionExpression &function, idx_t } } - if (try_bind_lambda) { - auto result = BindLambdaFunction(function, func->Cast(), depth); - if (!result.HasError()) { - // Lambda bind successful - return result; - } + // other scalar function + if (!try_bind_lambda) { + return BindFunction(function, func->Cast(), depth); } - // other scalar function - return BindFunction(function, func->Cast(), depth); + auto lambda_bind_result = BindLambdaFunction(function, func->Cast(), depth); + if (!lambda_bind_result.HasError()) { + return lambda_bind_result; + } + + auto json_bind_result = BindFunction(function, func->Cast(), depth); + if (!json_bind_result.HasError()) { + return json_bind_result; + } + + return BindResult("failed to bind function, either: " + lambda_bind_result.error + + "\n" + " or: " + + json_bind_result.error); } case CatalogType::MACRO_ENTRY: // macro function @@ -139,15 +148,25 @@ BindResult ExpressionBinder::BindFunction(FunctionExpression &function, ScalarFu BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, ScalarFunctionCatalogEntry &func, idx_t depth) { - // bind the children of the function expression - string error; + // scalar functions with lambdas can never be overloaded + if (func.functions.functions.size() != 1) { + return BindResult("This scalar function does not support lambdas!"); + } + + // get the callback function for the lambda parameter types + auto &scalar_function = func.functions.functions.front(); + auto &bind_lambda_function = scalar_function.bind_lambda; + if (!bind_lambda_function) { + return BindResult("This scalar function does not support lambdas!"); + } if (function.children.size() != 2) { - return BindResult("Invalid function arguments!"); + return BindResult("Invalid number of function arguments!"); } D_ASSERT(function.children[1]->GetExpressionClass() == ExpressionClass::LAMBDA); // bind the list parameter + string error; BindChild(function.children[0], depth, error); if (!error.empty()) { return BindResult(error); @@ -155,36 +174,39 @@ BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, Sc // get the logical type of the children of the list auto &list_child = BoundExpression::GetExpression(*function.children[0]); - if (list_child->return_type.id() != LogicalTypeId::LIST && list_child->return_type.id() != LogicalTypeId::SQLNULL && + if (list_child->return_type.id() != LogicalTypeId::LIST && list_child->return_type.id() != LogicalTypeId::ARRAY && + list_child->return_type.id() != LogicalTypeId::SQLNULL && list_child->return_type.id() != LogicalTypeId::UNKNOWN) { - return BindResult(" Invalid LIST argument to " + function.function_name + "!"); + return BindResult("Invalid LIST argument during lambda function binding!"); } LogicalType list_child_type = list_child->return_type.id(); if (list_child->return_type.id() != LogicalTypeId::SQLNULL && list_child->return_type.id() != LogicalTypeId::UNKNOWN) { - list_child_type = ListType::GetChildType(list_child->return_type); + + if (list_child->return_type.id() == LogicalTypeId::ARRAY) { + list_child_type = ArrayType::GetChildType(list_child->return_type); + } else { + list_child_type = ListType::GetChildType(list_child->return_type); + } } // bind the lambda parameter auto &lambda_expr = function.children[1]->Cast(); - BindResult bind_lambda_result = BindExpression(lambda_expr, depth, true, list_child_type); + BindResult bind_lambda_result = BindExpression(lambda_expr, depth, list_child_type, &bind_lambda_function); if (bind_lambda_result.HasError()) { - error = bind_lambda_result.error; - } else { - // successfully bound: replace the node with a BoundExpression - auto alias = function.children[1]->alias; - bind_lambda_result.expression->alias = alias; - if (!alias.empty()) { - bind_lambda_result.expression->alias = alias; - } - function.children[1] = make_uniq(std::move(bind_lambda_result.expression)); + return BindResult(bind_lambda_result.error); } - if (!error.empty()) { - return BindResult(error); + // successfully bound: replace the node with a BoundExpression + auto alias = function.children[1]->alias; + bind_lambda_result.expression->alias = alias; + if (!alias.empty()) { + bind_lambda_result.expression->alias = alias; } + function.children[1] = make_uniq(std::move(bind_lambda_result.expression)); + if (binder.GetBindingMode() == BindingMode::EXTRACT_NAMES) { return BindResult(make_uniq(Value(LogicalType::SQLNULL))); } @@ -199,7 +221,7 @@ BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, Sc // capture the (lambda) columns auto &bound_lambda_expr = children.back()->Cast(); - CaptureLambdaColumns(bound_lambda_expr.captures, list_child_type, bound_lambda_expr.lambda_expr); + CaptureLambdaColumns(bound_lambda_expr, bound_lambda_expr.lambda_expr, &bind_lambda_function, list_child_type); FunctionBinder function_binder(context); unique_ptr result = @@ -216,24 +238,26 @@ BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, Sc bound_function_expr.children.pop_back(); auto &bound_lambda = lambda->Cast(); - // push back (in reverse order) any nested lambda parameters so that we can later use them in the lambda expression - // (rhs) + // push back (in reverse order) any nested lambda parameters so that we can later use them in the lambda + // expression (rhs). This happens after we bound the lambda expression of this depth. So it is relevant for + // correctly binding lambdas one level 'out'. Therefore, the current parameter count does not matter here. + idx_t offset = 0; if (lambda_bindings) { for (idx_t i = lambda_bindings->size(); i > 0; i--) { - idx_t lambda_index = lambda_bindings->size() - i + 1; auto &binding = (*lambda_bindings)[i - 1]; + D_ASSERT(binding.names.size() == binding.types.size()); - D_ASSERT(binding.names.size() == 1); - D_ASSERT(binding.types.size() == 1); - - auto bound_lambda_param = - make_uniq(binding.names[0], binding.types[0], lambda_index); - bound_function_expr.children.push_back(std::move(bound_lambda_param)); + for (idx_t column_idx = binding.names.size(); column_idx > 0; column_idx--) { + auto bound_lambda_param = make_uniq(binding.names[column_idx - 1], + binding.types[column_idx - 1], offset); + offset++; + bound_function_expr.children.push_back(std::move(bound_lambda_param)); + } } } - // push back the captures into the children vector and the correct return types into the bound_function arguments + // push back the captures into the children vector for (auto &capture : bound_lambda.captures) { bound_function_expr.children.push_back(std::move(capture)); } diff --git a/src/duckdb/src/planner/binder/expression/bind_lambda.cpp b/src/duckdb/src/planner/binder/expression/bind_lambda.cpp index 56bb87edc..3d279dd99 100644 --- a/src/duckdb/src/planner/binder/expression/bind_lambda.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_lambda.cpp @@ -12,11 +12,34 @@ namespace duckdb { -BindResult ExpressionBinder::BindExpression(LambdaExpression &expr, idx_t depth, const bool is_lambda, - const LogicalType &list_child_type) { +idx_t GetLambdaParamCount(const vector &lambda_bindings) { + idx_t count = 0; + for (auto &binding : lambda_bindings) { + count += binding.names.size(); + } + return count; +} + +idx_t GetLambdaParamIndex(const vector &lambda_bindings, const BoundLambdaExpression &bound_lambda_expr, + const BoundLambdaRefExpression &bound_lambda_ref_expr) { + D_ASSERT(bound_lambda_ref_expr.lambda_idx < lambda_bindings.size()); + idx_t offset = 0; + // count the remaining lambda parameters BEFORE the current lambda parameter, + // as these will be in front of the current lambda parameter in the input chunk + for (idx_t i = bound_lambda_ref_expr.lambda_idx + 1; i < lambda_bindings.size(); i++) { + offset += lambda_bindings[i].names.size(); + } + offset += + lambda_bindings[bound_lambda_ref_expr.lambda_idx].names.size() - bound_lambda_ref_expr.binding.column_index - 1; + offset += bound_lambda_expr.parameter_count; + return offset; +} - if (!is_lambda) { - // this is for binding JSON +BindResult ExpressionBinder::BindExpression(LambdaExpression &expr, idx_t depth, const LogicalType &list_child_type, + optional_ptr bind_lambda_function) { + + // this is for binding JSON + if (!bind_lambda_function) { auto lhs_expr = expr.lhs->Copy(); OperatorExpression arrow_expr(ExpressionType::ARROW, std::move(lhs_expr), expr.expr->Copy()); return BindExpression(arrow_expr, depth); @@ -49,15 +72,15 @@ BindResult ExpressionBinder::BindExpression(LambdaExpression &expr, idx_t depth, // positional parameters as column references for (idx_t i = 0; i < expr.params.size(); i++) { if (expr.params[i]->GetExpressionClass() != ExpressionClass::COLUMN_REF) { - throw BinderException("Parameter must be a column name."); + throw BinderException("Lambda parameter must be a column name."); } auto column_ref = expr.params[i]->Cast(); if (column_ref.IsQualified()) { - throw BinderException("Invalid parameter name '%s': must be unqualified", column_ref.ToString()); + throw BinderException("Invalid lambda parameter name '%s': must be unqualified", column_ref.ToString()); } - column_types.emplace_back(list_child_type); + column_types.push_back((*bind_lambda_function)(i, list_child_type)); column_names.push_back(column_ref.GetColumnName()); params_strings.push_back(expr.params[i]->ToString()); } @@ -103,57 +126,77 @@ BindResult ExpressionBinder::BindExpression(LambdaExpression &expr, idx_t depth, void ExpressionBinder::TransformCapturedLambdaColumn(unique_ptr &original, unique_ptr &replacement, - vector> &captures, - LogicalType &list_child_type) { + BoundLambdaExpression &bound_lambda_expr, + const optional_ptr bind_lambda_function, + const LogicalType &list_child_type) { // check if the original expression is a lambda parameter if (original->expression_class == ExpressionClass::BOUND_LAMBDA_REF) { - // determine if this is the lambda parameter auto &bound_lambda_ref = original->Cast(); auto alias = bound_lambda_ref.alias; - if (lambda_bindings && bound_lambda_ref.lambda_index != lambda_bindings->size()) { + // refers to a lambda parameter outside the current lambda function + // so the lambda parameter will be inside the lambda_bindings + if (lambda_bindings && bound_lambda_ref.lambda_idx != lambda_bindings->size()) { - D_ASSERT(bound_lambda_ref.lambda_index < lambda_bindings->size()); - auto &lambda_binding = (*lambda_bindings)[bound_lambda_ref.lambda_index]; + auto &binding = (*lambda_bindings)[bound_lambda_ref.lambda_idx]; + D_ASSERT(binding.names.size() == binding.types.size()); - D_ASSERT(lambda_binding.names.size() == 1); - D_ASSERT(lambda_binding.types.size() == 1); - // refers to a lambda parameter outside of the current lambda function - replacement = - make_uniq(lambda_binding.names[0], lambda_binding.types[0], - lambda_bindings->size() - bound_lambda_ref.lambda_index + 1); + // find the matching dummy column in the lambda binding + for (idx_t column_idx = 0; column_idx < binding.names.size(); column_idx++) { + if (column_idx == bound_lambda_ref.binding.column_index) { - } else { - // refers to current lambda parameter - replacement = make_uniq(alias, list_child_type, 0); - } + // now create the replacement + auto index = GetLambdaParamIndex(*lambda_bindings, bound_lambda_expr, bound_lambda_ref); + replacement = make_uniq(binding.names[column_idx], + binding.types[column_idx], index); + return; + } + } - } else { - // always at least the current lambda parameter - idx_t index_offset = 1; - if (lambda_bindings) { - index_offset += lambda_bindings->size(); + // error resolving the lambda index + throw InternalException("Failed to bind lambda parameter internally"); } - // this is not a lambda parameter, so we need to create a new argument for the arguments vector - replacement = make_uniq(original->alias, original->return_type, - captures.size() + index_offset + 1); - captures.push_back(std::move(original)); + // refers to a lambda parameter inside the current lambda function + auto logical_type = (*bind_lambda_function)(bound_lambda_ref.binding.column_index, list_child_type); + auto index = bound_lambda_expr.parameter_count - bound_lambda_ref.binding.column_index - 1; + replacement = make_uniq(alias, logical_type, index); + return; + } + + // this is not a lambda parameter, get the capture offset + idx_t offset = 0; + if (lambda_bindings) { + offset += GetLambdaParamCount(*lambda_bindings); } + offset += bound_lambda_expr.parameter_count; + offset += bound_lambda_expr.captures.size(); + + replacement = make_uniq(original->alias, original->return_type, offset); + bound_lambda_expr.captures.push_back(std::move(original)); } -void ExpressionBinder::CaptureLambdaColumns(vector> &captures, LogicalType &list_child_type, - unique_ptr &expr) { +void ExpressionBinder::CaptureLambdaColumns(BoundLambdaExpression &bound_lambda_expr, unique_ptr &expr, + const optional_ptr bind_lambda_function, + const LogicalType &list_child_type) { if (expr->expression_class == ExpressionClass::BOUND_SUBQUERY) { throw InvalidInputException("Subqueries are not supported in lambda expressions!"); } + // these are bound depth-first + D_ASSERT(expr->expression_class != ExpressionClass::BOUND_LAMBDA); + + // we do not need to replace anything, as these will be constant in the lambda expression + // when executed by the expression executor + if (expr->expression_class == ExpressionClass::BOUND_CONSTANT) { + return; + } + // these expression classes do not have children, transform them - if (expr->expression_class == ExpressionClass::BOUND_CONSTANT || - expr->expression_class == ExpressionClass::BOUND_COLUMN_REF || + if (expr->expression_class == ExpressionClass::BOUND_COLUMN_REF || expr->expression_class == ExpressionClass::BOUND_PARAMETER || expr->expression_class == ExpressionClass::BOUND_LAMBDA_REF) { @@ -161,15 +204,16 @@ void ExpressionBinder::CaptureLambdaColumns(vector> &capt auto original = std::move(expr); unique_ptr replacement; - TransformCapturedLambdaColumn(original, replacement, captures, list_child_type); + TransformCapturedLambdaColumn(original, replacement, bound_lambda_expr, bind_lambda_function, list_child_type); // replace the expression expr = std::move(replacement); } else { // recursively enumerate the children of the expression - ExpressionIterator::EnumerateChildren( - *expr, [&](unique_ptr &child) { CaptureLambdaColumns(captures, list_child_type, child); }); + ExpressionIterator::EnumerateChildren(*expr, [&](unique_ptr &child) { + CaptureLambdaColumns(bound_lambda_expr, child, bind_lambda_function, list_child_type); + }); } expr->Verify(); diff --git a/src/duckdb/src/planner/binder/expression/bind_window_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_window_expression.cpp index 6a59ea4ab..c8a878344 100644 --- a/src/duckdb/src/planner/binder/expression/bind_window_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_window_expression.cpp @@ -289,6 +289,7 @@ BindResult BaseSelectBinder::BindWindow(WindowExpression &window, idx_t depth) { result->default_expr = CastWindowExpression(window.default_expr, result->return_type); result->start = window.start; result->end = window.end; + result->exclude_clause = window.exclude_clause; // create a BoundColumnRef that references this entry auto colref = make_uniq(std::move(name), result->return_type, diff --git a/src/duckdb/src/planner/binder/query_node/bind_setop_node.cpp b/src/duckdb/src/planner/binder/query_node/bind_setop_node.cpp index 5d5a09920..7b2c33752 100644 --- a/src/duckdb/src/planner/binder/query_node/bind_setop_node.cpp +++ b/src/duckdb/src/planner/binder/query_node/bind_setop_node.cpp @@ -51,7 +51,17 @@ static void GatherAliases(BoundQueryNode &node, case_insensitive_map_t &a idx_t index = reorder_idx[i]; - if (entry == aliases.end()) { + if (entry != aliases.end()) { + // the alias already exists + // check if there is a conflict + + if (entry->second != index) { + // there is a conflict + // we place "-1" in the aliases map at this location + // "-1" signifies that there is an ambiguous reference + aliases[name] = DConstants::INVALID_INDEX; + } + } else { // the alias is not in there yet, just assign it aliases[name] = index; } diff --git a/src/duckdb/src/planner/binder/statement/bind_create.cpp b/src/duckdb/src/planner/binder/statement/bind_create.cpp index 1a5722288..96bee4953 100644 --- a/src/duckdb/src/planner/binder/statement/bind_create.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_create.cpp @@ -214,6 +214,13 @@ void Binder::BindLogicalType(ClientContext &context, LogicalType &type, optional auto alias = type.GetAlias(); type = LogicalType::STRUCT(child_types); type.SetAlias(alias); + } else if (type.id() == LogicalTypeId::ARRAY) { + auto child_type = ArrayType::GetChildType(type); + auto array_size = ArrayType::GetSize(type); + BindLogicalType(context, child_type, catalog, schema); + auto alias = type.GetAlias(); + type = LogicalType::ARRAY(child_type, array_size); + type.SetAlias(alias); } else if (type.id() == LogicalTypeId::UNION) { auto member_types = UnionType::CopyMemberTypes(type); for (auto &member_type : member_types) { diff --git a/src/duckdb/src/planner/expression/bound_cast_expression.cpp b/src/duckdb/src/planner/expression/bound_cast_expression.cpp index 550577877..1eca9b3f4 100644 --- a/src/duckdb/src/planner/expression/bound_cast_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_cast_expression.cpp @@ -102,6 +102,14 @@ unique_ptr BoundCastExpression::AddCastToType(ClientContext &context return AddCastToTypeInternal(std::move(expr), target_type, cast_functions, get_input, try_cast); } +unique_ptr BoundCastExpression::AddArrayCastToList(ClientContext &context, unique_ptr expr) { + if (expr->return_type.id() != LogicalTypeId::ARRAY) { + return expr; + } + auto &child_type = ArrayType::GetChildType(expr->return_type); + return BoundCastExpression::AddCastToType(context, std::move(expr), LogicalType::LIST(child_type)); +} + bool BoundCastExpression::CastIsInvertible(const LogicalType &source_type, const LogicalType &target_type) { D_ASSERT(source_type.IsValid() && target_type.IsValid()); if (source_type.id() == LogicalTypeId::BOOLEAN || target_type.id() == LogicalTypeId::BOOLEAN) { diff --git a/src/duckdb/src/planner/expression/bound_function_expression.cpp b/src/duckdb/src/planner/expression/bound_function_expression.cpp index 40ba170c6..4c665677e 100644 --- a/src/duckdb/src/planner/expression/bound_function_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_function_expression.cpp @@ -5,6 +5,7 @@ #include "duckdb/function/function_serialization.hpp" #include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/core_functions/lambda_functions.hpp" namespace duckdb { @@ -23,6 +24,17 @@ bool BoundFunctionExpression::HasSideEffects() const { bool BoundFunctionExpression::IsFoldable() const { // functions with side effects cannot be folded: they have to be executed once for every row + if (function.bind_lambda) { + // This is a lambda function + D_ASSERT(bind_info); + auto &lambda_bind_data = bind_info->Cast(); + if (lambda_bind_data.lambda_expr) { + auto &expr = *lambda_bind_data.lambda_expr; + if (expr.HasSideEffects()) { + return false; + } + } + } return function.side_effects == FunctionSideEffects::HAS_SIDE_EFFECTS ? false : Expression::IsFoldable(); } diff --git a/src/duckdb/src/planner/expression/bound_lambdaref_expression.cpp b/src/duckdb/src/planner/expression/bound_lambdaref_expression.cpp index 388404cf0..8fcf1b67e 100644 --- a/src/duckdb/src/planner/expression/bound_lambdaref_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_lambdaref_expression.cpp @@ -6,24 +6,24 @@ namespace duckdb { BoundLambdaRefExpression::BoundLambdaRefExpression(string alias_p, LogicalType type, ColumnBinding binding, - idx_t lambda_index, idx_t depth) + idx_t lambda_idx, idx_t depth) : Expression(ExpressionType::BOUND_LAMBDA_REF, ExpressionClass::BOUND_LAMBDA_REF, std::move(type)), - binding(binding), lambda_index(lambda_index), depth(depth) { + binding(binding), lambda_idx(lambda_idx), depth(depth) { this->alias = std::move(alias_p); } -BoundLambdaRefExpression::BoundLambdaRefExpression(LogicalType type, ColumnBinding binding, idx_t lambda_index, +BoundLambdaRefExpression::BoundLambdaRefExpression(LogicalType type, ColumnBinding binding, idx_t lambda_idx, idx_t depth) - : BoundLambdaRefExpression(string(), std::move(type), binding, lambda_index, depth) { + : BoundLambdaRefExpression(string(), std::move(type), binding, lambda_idx, depth) { } unique_ptr BoundLambdaRefExpression::Copy() { - return make_uniq(alias, return_type, binding, lambda_index, depth); + return make_uniq(alias, return_type, binding, lambda_idx, depth); } hash_t BoundLambdaRefExpression::Hash() const { auto result = Expression::Hash(); - result = CombineHash(result, duckdb::Hash(lambda_index)); + result = CombineHash(result, duckdb::Hash(lambda_idx)); result = CombineHash(result, duckdb::Hash(binding.column_index)); result = CombineHash(result, duckdb::Hash(binding.table_index)); return CombineHash(result, duckdb::Hash(depth)); @@ -34,15 +34,14 @@ bool BoundLambdaRefExpression::Equals(const BaseExpression &other_p) const { return false; } auto &other = other_p.Cast(); - return other.binding == binding && other.lambda_index == lambda_index && other.depth == depth; + return other.binding == binding && other.lambda_idx == lambda_idx && other.depth == depth; } string BoundLambdaRefExpression::ToString() const { if (!alias.empty()) { return alias; } - return "#[" + to_string(binding.table_index) + "." + to_string(binding.column_index) + "." + - to_string(lambda_index) + "]"; + return "#[" + to_string(binding.table_index) + "." + to_string(binding.column_index) + "." + to_string(lambda_idx) + + "]"; } - } // namespace duckdb diff --git a/src/duckdb/src/planner/expression/bound_window_expression.cpp b/src/duckdb/src/planner/expression/bound_window_expression.cpp index 684c7ef79..58826f587 100644 --- a/src/duckdb/src/planner/expression/bound_window_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_window_expression.cpp @@ -33,6 +33,9 @@ bool BoundWindowExpression::Equals(const BaseExpression &other_p) const { if (start != other.start || end != other.end) { return false; } + if (exclude_clause != other.exclude_clause) { + return false; + } // check if the child expressions are equivalent if (!Expression::ListEquals(children, other.children)) { return false; @@ -99,6 +102,7 @@ unique_ptr BoundWindowExpression::Copy() { new_window->start = start; new_window->end = end; + new_window->exclude_clause = exclude_clause; new_window->start_expr = start_expr ? start_expr->Copy() : nullptr; new_window->end_expr = end_expr ? end_expr->Copy() : nullptr; new_window->offset_expr = offset_expr ? offset_expr->Copy() : nullptr; @@ -126,6 +130,7 @@ void BoundWindowExpression::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault(209, "end_expr", end_expr, unique_ptr()); serializer.WritePropertyWithDefault(210, "offset_expr", offset_expr, unique_ptr()); serializer.WritePropertyWithDefault(211, "default_expr", default_expr, unique_ptr()); + serializer.WriteProperty(212, "exclude_clause", exclude_clause); } unique_ptr BoundWindowExpression::Deserialize(Deserializer &deserializer) { @@ -153,6 +158,7 @@ unique_ptr BoundWindowExpression::Deserialize(Deserializer &deserial deserializer.ReadPropertyWithDefault(209, "end_expr", result->end_expr, unique_ptr()); deserializer.ReadPropertyWithDefault(210, "offset_expr", result->offset_expr, unique_ptr()); deserializer.ReadPropertyWithDefault(211, "default_expr", result->default_expr, unique_ptr()); + deserializer.ReadProperty(212, "exclude_clause", result->exclude_clause); return std::move(result); } diff --git a/src/duckdb/src/planner/expression_binder.cpp b/src/duckdb/src/planner/expression_binder.cpp index 266b4102c..2560cd21d 100644 --- a/src/duckdb/src/planner/expression_binder.cpp +++ b/src/duckdb/src/planner/expression_binder.cpp @@ -61,6 +61,8 @@ BindResult ExpressionBinder::BindExpression(unique_ptr &expr, return BindExpression(expr_ref.Cast(), depth); case ExpressionClass::COLUMN_REF: return BindExpression(expr_ref.Cast(), depth); + case ExpressionClass::LAMBDA_REF: + return BindExpression(expr_ref.Cast(), depth); case ExpressionClass::COMPARISON: return BindExpression(expr_ref.Cast(), depth); case ExpressionClass::CONJUNCTION: @@ -77,7 +79,7 @@ BindResult ExpressionBinder::BindExpression(unique_ptr &expr, return BindExpression(function, depth, expr); } case ExpressionClass::LAMBDA: - return BindExpression(expr_ref.Cast(), depth, false, LogicalTypeId::INVALID); + return BindExpression(expr_ref.Cast(), depth, LogicalTypeId::INVALID, nullptr); case ExpressionClass::OPERATOR: return BindExpression(expr_ref.Cast(), depth); case ExpressionClass::SUBQUERY: @@ -166,6 +168,8 @@ bool ExpressionBinder::ContainsType(const LogicalType &type, LogicalTypeId targe case LogicalTypeId::LIST: case LogicalTypeId::MAP: return ContainsType(ListType::GetChildType(type), target); + case LogicalTypeId::ARRAY: + return ContainsType(ArrayType::GetChildType(type), target); default: return false; } @@ -195,6 +199,9 @@ LogicalType ExpressionBinder::ExchangeType(const LogicalType &type, LogicalTypeI return LogicalType::LIST(ExchangeType(ListType::GetChildType(type), target, new_type)); case LogicalTypeId::MAP: return LogicalType::MAP(ExchangeType(ListType::GetChildType(type), target, new_type)); + case LogicalTypeId::ARRAY: + return LogicalType::ARRAY(ExchangeType(ArrayType::GetChildType(type), target, new_type), + ArrayType::GetSize(type)); default: return type; } diff --git a/src/duckdb/src/planner/expression_binder/table_function_binder.cpp b/src/duckdb/src/planner/expression_binder/table_function_binder.cpp index 30010b1ba..f6f749585 100644 --- a/src/duckdb/src/planner/expression_binder/table_function_binder.cpp +++ b/src/duckdb/src/planner/expression_binder/table_function_binder.cpp @@ -8,6 +8,12 @@ namespace duckdb { TableFunctionBinder::TableFunctionBinder(Binder &binder, ClientContext &context) : ExpressionBinder(binder, context) { } +BindResult TableFunctionBinder::BindLambdaReference(LambdaRefExpression &expr, idx_t depth) { + D_ASSERT(lambda_bindings && expr.lambda_idx < lambda_bindings->size()); + auto &lambdaref = expr.Cast(); + return (*lambda_bindings)[expr.lambda_idx].Bind(lambdaref, depth); +} + BindResult TableFunctionBinder::BindColumnReference(ColumnRefExpression &expr, idx_t depth, bool root_expression) { // if this is a lambda parameters, then we temporarily add a BoundLambdaRef, @@ -15,11 +21,13 @@ BindResult TableFunctionBinder::BindColumnReference(ColumnRefExpression &expr, i if (lambda_bindings) { auto &colref = expr.Cast(); for (idx_t i = 0; i < lambda_bindings->size(); i++) { - if (colref.GetColumnName() == (*lambda_bindings)[i].dummy_name) { - return (*lambda_bindings)[i].Bind(colref, i, depth); + if ((*lambda_bindings)[i].HasMatchingBinding(colref.GetColumnName())) { + auto lambdaref = make_uniq(i, colref.GetColumnName()); + return BindLambdaReference(*lambdaref, depth); } } } + auto value_function = ExpressionBinder::GetSQLValueFunction(expr.GetColumnName()); if (value_function) { return BindExpression(value_function, depth, root_expression); @@ -33,6 +41,8 @@ BindResult TableFunctionBinder::BindExpression(unique_ptr &exp bool root_expression) { auto &expr = *expr_ptr; switch (expr.GetExpressionClass()) { + case ExpressionClass::LAMBDA_REF: + return BindLambdaReference(expr.Cast(), depth); case ExpressionClass::COLUMN_REF: return BindColumnReference(expr.Cast(), depth, root_expression); case ExpressionClass::SUBQUERY: diff --git a/src/duckdb/src/planner/table_binding.cpp b/src/duckdb/src/planner/table_binding.cpp index aec069f3c..3f1d572e7 100644 --- a/src/duckdb/src/planner/table_binding.cpp +++ b/src/duckdb/src/planner/table_binding.cpp @@ -222,10 +222,10 @@ string TableBinding::ColumnNotFoundError(const string &column_name) const { return StringUtil::Format("Table \"%s\" does not have a column named \"%s\"", alias, column_name); } -DummyBinding::DummyBinding(vector types_p, vector names_p, string dummy_name_p) - : Binding(BindingType::DUMMY, DummyBinding::DUMMY_NAME + dummy_name_p, std::move(types_p), std::move(names_p), +DummyBinding::DummyBinding(vector types, vector names, string dummy_name) + : Binding(BindingType::DUMMY, DummyBinding::DUMMY_NAME + dummy_name, std::move(types), std::move(names), DConstants::INVALID_INDEX), - dummy_name(std::move(dummy_name_p)) { + dummy_name(std::move(dummy_name)) { } BindResult DummyBinding::Bind(ColumnRefExpression &colref, idx_t depth) { @@ -239,14 +239,14 @@ BindResult DummyBinding::Bind(ColumnRefExpression &colref, idx_t depth) { return BindResult(make_uniq(colref.GetName(), types[column_index], binding, depth)); } -BindResult DummyBinding::Bind(ColumnRefExpression &colref, idx_t lambda_index, idx_t depth) { +BindResult DummyBinding::Bind(LambdaRefExpression &lambdaref, idx_t depth) { column_t column_index; - if (!TryGetBindingIndex(colref.GetColumnName(), column_index)) { - throw InternalException("Column %s not found in bindings", colref.GetColumnName()); + if (!TryGetBindingIndex(lambdaref.GetName(), column_index)) { + throw InternalException("Column %s not found in bindings", lambdaref.GetName()); } ColumnBinding binding(index, column_index); - return BindResult( - make_uniq(colref.GetName(), types[column_index], binding, lambda_index, depth)); + return BindResult(make_uniq(lambdaref.GetName(), types[column_index], binding, + lambdaref.lambda_idx, depth)); } unique_ptr DummyBinding::ParamToArg(ColumnRefExpression &colref) { diff --git a/src/duckdb/src/storage/serialization/serialize_expression.cpp b/src/duckdb/src/storage/serialization/serialize_expression.cpp index bc0b6e191..e26e4ca49 100644 --- a/src/duckdb/src/storage/serialization/serialize_expression.cpp +++ b/src/duckdb/src/storage/serialization/serialize_expression.cpp @@ -213,16 +213,16 @@ void BoundLambdaRefExpression::Serialize(Serializer &serializer) const { Expression::Serialize(serializer); serializer.WriteProperty(200, "return_type", return_type); serializer.WriteProperty(201, "binding", binding); - serializer.WritePropertyWithDefault(202, "lambda_index", lambda_index); + serializer.WritePropertyWithDefault(202, "lambda_index", lambda_idx); serializer.WritePropertyWithDefault(203, "depth", depth); } unique_ptr BoundLambdaRefExpression::Deserialize(Deserializer &deserializer) { auto return_type = deserializer.ReadProperty(200, "return_type"); auto binding = deserializer.ReadProperty(201, "binding"); - auto lambda_index = deserializer.ReadPropertyWithDefault(202, "lambda_index"); + auto lambda_idx = deserializer.ReadPropertyWithDefault(202, "lambda_index"); auto depth = deserializer.ReadPropertyWithDefault(203, "depth"); - auto result = duckdb::unique_ptr(new BoundLambdaRefExpression(std::move(return_type), binding, lambda_index, depth)); + auto result = duckdb::unique_ptr(new BoundLambdaRefExpression(std::move(return_type), binding, lambda_idx, depth)); return std::move(result); } diff --git a/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp b/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp index 39297311d..aee88ffd2 100644 --- a/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp +++ b/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp @@ -55,6 +55,9 @@ unique_ptr ParsedExpression::Deserialize(Deserializer &deseria case ExpressionClass::LAMBDA: result = LambdaExpression::Deserialize(deserializer); break; + case ExpressionClass::LAMBDA_REF: + result = LambdaRefExpression::Deserialize(deserializer); + break; case ExpressionClass::OPERATOR: result = OperatorExpression::Deserialize(deserializer); break; @@ -233,6 +236,19 @@ unique_ptr LambdaExpression::Deserialize(Deserializer &deseria return std::move(result); } +void LambdaRefExpression::Serialize(Serializer &serializer) const { + ParsedExpression::Serialize(serializer); + serializer.WritePropertyWithDefault(200, "lambda_idx", lambda_idx); + serializer.WritePropertyWithDefault(201, "column_name", column_name); +} + +unique_ptr LambdaRefExpression::Deserialize(Deserializer &deserializer) { + auto lambda_idx = deserializer.ReadPropertyWithDefault(200, "lambda_idx"); + auto column_name = deserializer.ReadPropertyWithDefault(201, "column_name"); + auto result = duckdb::unique_ptr(new LambdaRefExpression(lambda_idx, std::move(column_name))); + return std::move(result); +} + void OperatorExpression::Serialize(Serializer &serializer) const { ParsedExpression::Serialize(serializer); serializer.WritePropertyWithDefault>>(200, "children", children); @@ -318,6 +334,7 @@ void WindowExpression::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault>(211, "default_expr", default_expr); serializer.WritePropertyWithDefault(212, "ignore_nulls", ignore_nulls); serializer.WritePropertyWithDefault>(213, "filter_expr", filter_expr); + serializer.WritePropertyWithDefault(214, "exclude_clause", exclude_clause, WindowExcludeMode::NO_OTHER); } unique_ptr WindowExpression::Deserialize(Deserializer &deserializer) { @@ -336,6 +353,7 @@ unique_ptr WindowExpression::Deserialize(Deserializer &deseria deserializer.ReadPropertyWithDefault>(211, "default_expr", result->default_expr); deserializer.ReadPropertyWithDefault(212, "ignore_nulls", result->ignore_nulls); deserializer.ReadPropertyWithDefault>(213, "filter_expr", result->filter_expr); + deserializer.ReadPropertyWithDefault(214, "exclude_clause", result->exclude_clause, WindowExcludeMode::NO_OTHER); return std::move(result); } diff --git a/src/duckdb/src/storage/serialization/serialize_types.cpp b/src/duckdb/src/storage/serialization/serialize_types.cpp index 13b5b0787..be7068223 100644 --- a/src/duckdb/src/storage/serialization/serialize_types.cpp +++ b/src/duckdb/src/storage/serialization/serialize_types.cpp @@ -22,6 +22,9 @@ shared_ptr ExtraTypeInfo::Deserialize(Deserializer &deserializer) case ExtraTypeInfoType::AGGREGATE_STATE_TYPE_INFO: result = AggregateStateTypeInfo::Deserialize(deserializer); break; + case ExtraTypeInfoType::ARRAY_TYPE_INFO: + result = ArrayTypeInfo::Deserialize(deserializer); + break; case ExtraTypeInfoType::DECIMAL_TYPE_INFO: result = DecimalTypeInfo::Deserialize(deserializer); break; @@ -67,6 +70,19 @@ shared_ptr AggregateStateTypeInfo::Deserialize(Deserializer &dese return std::move(result); } +void ArrayTypeInfo::Serialize(Serializer &serializer) const { + ExtraTypeInfo::Serialize(serializer); + serializer.WriteProperty(200, "child_type", child_type); + serializer.WritePropertyWithDefault(201, "size", size); +} + +shared_ptr ArrayTypeInfo::Deserialize(Deserializer &deserializer) { + auto child_type = deserializer.ReadProperty(200, "child_type"); + auto size = deserializer.ReadPropertyWithDefault(201, "size"); + auto result = duckdb::shared_ptr(new ArrayTypeInfo(std::move(child_type), size)); + return std::move(result); +} + void DecimalTypeInfo::Serialize(Serializer &serializer) const { ExtraTypeInfo::Serialize(serializer); serializer.WritePropertyWithDefault(200, "width", width); diff --git a/src/duckdb/src/storage/statistics/array_stats.cpp b/src/duckdb/src/storage/statistics/array_stats.cpp new file mode 100644 index 000000000..d01b573ba --- /dev/null +++ b/src/duckdb/src/storage/statistics/array_stats.cpp @@ -0,0 +1,131 @@ +#include "duckdb/storage/statistics/array_stats.hpp" +#include "duckdb/storage/statistics/base_statistics.hpp" +#include "duckdb/common/string_util.hpp" +#include "duckdb/common/types/vector.hpp" +#include "duckdb/common/serializer/serializer.hpp" +#include "duckdb/common/serializer/deserializer.hpp" + +namespace duckdb { + +void ArrayStats::Construct(BaseStatistics &stats) { + stats.child_stats = unsafe_unique_array(new BaseStatistics[1]); + BaseStatistics::Construct(stats.child_stats[0], ArrayType::GetChildType(stats.GetType())); +} + +BaseStatistics ArrayStats::CreateUnknown(LogicalType type) { + auto &child_type = ArrayType::GetChildType(type); + BaseStatistics result(std::move(type)); + result.InitializeUnknown(); + result.child_stats[0].Copy(BaseStatistics::CreateUnknown(child_type)); + return result; +} + +BaseStatistics ArrayStats::CreateEmpty(LogicalType type) { + auto &child_type = ArrayType::GetChildType(type); + BaseStatistics result(std::move(type)); + result.InitializeEmpty(); + result.child_stats[0].Copy(BaseStatistics::CreateEmpty(child_type)); + return result; +} + +void ArrayStats::Copy(BaseStatistics &stats, const BaseStatistics &other) { + D_ASSERT(stats.child_stats); + D_ASSERT(other.child_stats); + stats.child_stats[0].Copy(other.child_stats[0]); +} + +const BaseStatistics &ArrayStats::GetChildStats(const BaseStatistics &stats) { + if (stats.GetStatsType() != StatisticsType::ARRAY_STATS) { + throw InternalException("ArrayStats::GetChildStats called on stats that is not a array"); + } + D_ASSERT(stats.child_stats); + return stats.child_stats[0]; +} +BaseStatistics &ArrayStats::GetChildStats(BaseStatistics &stats) { + if (stats.GetStatsType() != StatisticsType::ARRAY_STATS) { + throw InternalException("ArrayStats::GetChildStats called on stats that is not a array"); + } + D_ASSERT(stats.child_stats); + return stats.child_stats[0]; +} + +void ArrayStats::SetChildStats(BaseStatistics &stats, unique_ptr new_stats) { + if (!new_stats) { + stats.child_stats[0].Copy(BaseStatistics::CreateUnknown(ArrayType::GetChildType(stats.GetType()))); + } else { + stats.child_stats[0].Copy(*new_stats); + } +} + +void ArrayStats::Merge(BaseStatistics &stats, const BaseStatistics &other) { + if (other.GetType().id() == LogicalTypeId::VALIDITY) { + return; + } + + auto &child_stats = ArrayStats::GetChildStats(stats); + auto &other_child_stats = ArrayStats::GetChildStats(other); + child_stats.Merge(other_child_stats); +} + +void ArrayStats::Serialize(const BaseStatistics &stats, Serializer &serializer) { + auto &child_stats = ArrayStats::GetChildStats(stats); + serializer.WriteProperty(200, "child_stats", child_stats); +} + +void ArrayStats::Deserialize(Deserializer &deserializer, BaseStatistics &base) { + auto &type = base.GetType(); + D_ASSERT(type.id() == LogicalTypeId::ARRAY); + auto &child_type = ArrayType::GetChildType(type); + + // Push the logical type of the child type to the deserialization context + deserializer.Set(const_cast(child_type)); + base.child_stats[0].Copy(deserializer.ReadProperty(200, "child_stats")); + deserializer.Unset(); +} + +string ArrayStats::ToString(const BaseStatistics &stats) { + auto &child_stats = ArrayStats::GetChildStats(stats); + return StringUtil::Format("[%s]", child_stats.ToString()); +} + +void ArrayStats::Verify(const BaseStatistics &stats, Vector &vector, const SelectionVector &sel, idx_t count) { + auto &child_stats = ArrayStats::GetChildStats(stats); + auto &child_entry = ArrayVector::GetEntry(vector); + auto array_size = ArrayType::GetSize(vector.GetType()); + + UnifiedVectorFormat vdata; + vector.ToUnifiedFormat(count, vdata); + + // Basically, + // 1. Count the number of valid arrays + // 2. Create a selection vector with the size of the number of valid arrays * array_size + // 3. Fill the selection vector with the offsets of all the elements in the child vector + // that exist in each valid array + // 4. Use that selection vector to verify the child stats + + idx_t valid_count = 0; + for (idx_t i = 0; i < count; i++) { + auto idx = sel.get_index(i); + auto index = vdata.sel->get_index(idx); + if (vdata.validity.RowIsValid(index)) { + valid_count++; + } + } + + SelectionVector element_sel(valid_count * array_size); + idx_t element_count = 0; + for (idx_t i = 0; i < count; i++) { + auto idx = sel.get_index(i); + auto index = vdata.sel->get_index(idx); + auto offset = index * array_size; + if (vdata.validity.RowIsValid(index)) { + for (idx_t elem_idx = 0; elem_idx < array_size; elem_idx++) { + element_sel.set_index(element_count++, offset + elem_idx); + } + } + } + + child_stats.Verify(child_entry, element_sel, element_count); +} + +} // namespace duckdb diff --git a/src/duckdb/src/storage/statistics/base_statistics.cpp b/src/duckdb/src/storage/statistics/base_statistics.cpp index fda77cf90..dfa389b3f 100644 --- a/src/duckdb/src/storage/statistics/base_statistics.cpp +++ b/src/duckdb/src/storage/statistics/base_statistics.cpp @@ -4,6 +4,7 @@ #include "duckdb/storage/statistics/base_statistics.hpp" #include "duckdb/storage/statistics/list_stats.hpp" #include "duckdb/storage/statistics/struct_stats.hpp" +#include "duckdb/storage/statistics/array_stats.hpp" #include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/serializer/deserializer.hpp" @@ -27,6 +28,9 @@ void BaseStatistics::Construct(BaseStatistics &stats, LogicalType type) { case StatisticsType::STRUCT_STATS: StructStats::Construct(stats); break; + case StatisticsType::ARRAY_STATS: + ArrayStats::Construct(stats); + break; default: break; } @@ -78,6 +82,8 @@ StatisticsType BaseStatistics::GetStatsType(const LogicalType &type) { return StatisticsType::STRUCT_STATS; case PhysicalType::LIST: return StatisticsType::LIST_STATS; + case PhysicalType::ARRAY: + return StatisticsType::ARRAY_STATS; case PhysicalType::BIT: case PhysicalType::INTERVAL: default: @@ -143,6 +149,9 @@ void BaseStatistics::Merge(const BaseStatistics &other) { case StatisticsType::STRUCT_STATS: StructStats::Merge(*this, other); break; + case StatisticsType::ARRAY_STATS: + ArrayStats::Merge(*this, other); + break; default: break; } @@ -162,6 +171,8 @@ BaseStatistics BaseStatistics::CreateUnknownType(LogicalType type) { return ListStats::CreateUnknown(std::move(type)); case StatisticsType::STRUCT_STATS: return StructStats::CreateUnknown(std::move(type)); + case StatisticsType::ARRAY_STATS: + return ArrayStats::CreateUnknown(std::move(type)); default: return BaseStatistics(std::move(type)); } @@ -177,6 +188,8 @@ BaseStatistics BaseStatistics::CreateEmptyType(LogicalType type) { return ListStats::CreateEmpty(std::move(type)); case StatisticsType::STRUCT_STATS: return StructStats::CreateEmpty(std::move(type)); + case StatisticsType::ARRAY_STATS: + return ArrayStats::CreateEmpty(std::move(type)); default: return BaseStatistics(std::move(type)); } @@ -213,6 +226,9 @@ void BaseStatistics::Copy(const BaseStatistics &other) { case StatisticsType::STRUCT_STATS: StructStats::Copy(*this, other); break; + case StatisticsType::ARRAY_STATS: + ArrayStats::Copy(*this, other); + break; default: break; } @@ -291,6 +307,8 @@ void BaseStatistics::Serialize(Serializer &serializer) const { case StatisticsType::STRUCT_STATS: StructStats::Serialize(*this, serializer); break; + case StatisticsType::ARRAY_STATS: + ArrayStats::Serialize(*this, serializer); default: break; } @@ -327,6 +345,9 @@ BaseStatistics BaseStatistics::Deserialize(Deserializer &deserializer) { case StatisticsType::STRUCT_STATS: StructStats::Deserialize(obj, stats); break; + case StatisticsType::ARRAY_STATS: + ArrayStats::Deserialize(obj, stats); + break; default: break; } @@ -354,6 +375,9 @@ string BaseStatistics::ToString() const { case StatisticsType::STRUCT_STATS: result = StructStats::ToString(*this) + result; break; + case StatisticsType::ARRAY_STATS: + result = ArrayStats::ToString(*this) + result; + break; default: break; } @@ -375,6 +399,9 @@ void BaseStatistics::Verify(Vector &vector, const SelectionVector &sel, idx_t co case StatisticsType::STRUCT_STATS: StructStats::Verify(*this, vector, sel, count); break; + case StatisticsType::ARRAY_STATS: + ArrayStats::Verify(*this, vector, sel, count); + break; default: break; } @@ -448,6 +475,17 @@ BaseStatistics BaseStatistics::FromConstantType(const Value &input) { } return result; } + case StatisticsType::ARRAY_STATS: { + auto result = ArrayStats::CreateEmpty(input.type()); + auto &child_stats = ArrayStats::GetChildStats(result); + if (!input.IsNull()) { + auto &list_children = ArrayValue::GetChildren(input); + for (auto &child_element : list_children) { + child_stats.Merge(FromConstant(child_element)); + } + } + return result; + } default: return BaseStatistics(input.type()); } diff --git a/src/duckdb/src/storage/statistics/distinct_statistics.cpp b/src/duckdb/src/storage/statistics/distinct_statistics.cpp index e80cc140e..7d4e6e92a 100644 --- a/src/duckdb/src/storage/statistics/distinct_statistics.cpp +++ b/src/duckdb/src/storage/statistics/distinct_statistics.cpp @@ -69,7 +69,9 @@ idx_t DistinctStatistics::GetCount() const { } bool DistinctStatistics::TypeIsSupported(const LogicalType &type) { - return type.InternalType() != PhysicalType::LIST && type.InternalType() != PhysicalType::STRUCT; + auto physical_type = type.InternalType(); + return physical_type != PhysicalType::LIST && physical_type != PhysicalType::STRUCT && + physical_type != PhysicalType::ARRAY; } } // namespace duckdb diff --git a/src/duckdb/src/storage/table/array_column_data.cpp b/src/duckdb/src/storage/table/array_column_data.cpp new file mode 100644 index 000000000..993e3cdd0 --- /dev/null +++ b/src/duckdb/src/storage/table/array_column_data.cpp @@ -0,0 +1,228 @@ +#include "duckdb/storage/table/array_column_data.hpp" +#include "duckdb/storage/statistics/array_stats.hpp" +#include "duckdb/common/serializer/serializer.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/storage/table/column_checkpoint_state.hpp" +#include "duckdb/storage/table/append_state.hpp" +#include "duckdb/storage/table/scan_state.hpp" + +namespace duckdb { + +ArrayColumnData::ArrayColumnData(BlockManager &block_manager, DataTableInfo &info, idx_t column_index, idx_t start_row, + LogicalType type_p, optional_ptr parent) + : ColumnData(block_manager, info, column_index, start_row, std::move(type_p), parent), + validity(block_manager, info, 0, start_row, *this) { + D_ASSERT(type.InternalType() == PhysicalType::ARRAY); + auto &child_type = ArrayType::GetChildType(type); + // the child column, with column index 1 (0 is the validity mask) + child_column = ColumnData::CreateColumnUnique(block_manager, info, 1, start_row, child_type, this); +} + +void ArrayColumnData::SetStart(idx_t new_start) { + this->start = new_start; + child_column->SetStart(new_start); + validity.SetStart(new_start); +} + +bool ArrayColumnData::CheckZonemap(ColumnScanState &state, TableFilter &filter) { + // FIXME: There is nothing preventing us from supporting this, but it's not implemented yet. + // table filters are not supported yet for fixed size list columns + return false; +} + +void ArrayColumnData::InitializeScan(ColumnScanState &state) { + // initialize the validity segment + D_ASSERT(state.child_states.size() == 2); + + state.row_index = 0; + state.current = nullptr; + + validity.InitializeScan(state.child_states[0]); + + // initialize the child scan + child_column->InitializeScan(state.child_states[1]); +} + +void ArrayColumnData::InitializeScanWithOffset(ColumnScanState &state, idx_t row_idx) { + D_ASSERT(state.child_states.size() == 2); + + if (row_idx == 0) { + // Trivial case, no offset + InitializeScan(state); + return; + } + + state.row_index = row_idx; + state.current = nullptr; + + // initialize the validity segment + validity.InitializeScanWithOffset(state.child_states[0], row_idx); + + auto array_size = ArrayType::GetSize(type); + auto child_offset = (row_idx - start) * array_size; + + D_ASSERT(child_offset <= child_column->GetMaxEntry()); + if (child_offset < child_column->GetMaxEntry()) { + child_column->InitializeScanWithOffset(state.child_states[1], start + child_offset); + } +} + +idx_t ArrayColumnData::Scan(TransactionData transaction, idx_t vector_index, ColumnScanState &state, Vector &result) { + return ScanCount(state, result, STANDARD_VECTOR_SIZE); +} + +idx_t ArrayColumnData::ScanCommitted(idx_t vector_index, ColumnScanState &state, Vector &result, bool allow_updates) { + return ScanCount(state, result, STANDARD_VECTOR_SIZE); +} + +idx_t ArrayColumnData::ScanCount(ColumnScanState &state, Vector &result, idx_t count) { + // Scan validity + auto scan_count = validity.ScanCount(state.child_states[0], result, count); + auto array_size = ArrayType::GetSize(type); + // Scan child column + auto &child_vec = ArrayVector::GetEntry(result); + child_column->ScanCount(state.child_states[1], child_vec, count * array_size); + return scan_count; +} + +void ArrayColumnData::Skip(ColumnScanState &state, idx_t count) { + // Skip validity + validity.Skip(state.child_states[0], count); + // Skip child column + auto array_size = ArrayType::GetSize(type); + child_column->Skip(state.child_states[1], count * array_size); +} + +void ArrayColumnData::InitializeAppend(ColumnAppendState &state) { + ColumnAppendState validity_append; + validity.InitializeAppend(validity_append); + state.child_appends.push_back(std::move(validity_append)); + + ColumnAppendState child_append; + child_column->InitializeAppend(child_append); + state.child_appends.push_back(std::move(child_append)); +} + +void ArrayColumnData::Append(BaseStatistics &stats, ColumnAppendState &state, Vector &vector, idx_t count) { + vector.Flatten(count); + // Append validity + validity.Append(stats, state.child_appends[0], vector, count); + // Append child column + auto array_size = ArrayType::GetSize(type); + auto &child_vec = ArrayVector::GetEntry(vector); + child_column->Append(ArrayStats::GetChildStats(stats), state.child_appends[1], child_vec, count * array_size); + + this->count += count; +} + +void ArrayColumnData::RevertAppend(row_t start_row) { + // Revert validity + validity.RevertAppend(start_row); + // Revert child column + auto array_size = ArrayType::GetSize(type); + child_column->RevertAppend(start_row * array_size); + + this->count = start_row - this->start; +} + +idx_t ArrayColumnData::Fetch(ColumnScanState &state, row_t row_id, Vector &result) { + throw NotImplementedException("Array Fetch"); +} + +void ArrayColumnData::Update(TransactionData transaction, idx_t column_index, Vector &update_vector, row_t *row_ids, + idx_t update_count) { + throw NotImplementedException("Array Update is not supported."); +} + +void ArrayColumnData::UpdateColumn(TransactionData transaction, const vector &column_path, + Vector &update_vector, row_t *row_ids, idx_t update_count, idx_t depth) { + throw NotImplementedException("Array Update Column is not supported"); +} + +unique_ptr ArrayColumnData::GetUpdateStatistics() { + return nullptr; +} + +void ArrayColumnData::FetchRow(TransactionData transaction, ColumnFetchState &state, row_t row_id, Vector &result, + idx_t result_idx) { + + // Create state for child column + state.child_states.push_back(make_uniq()); + + // Fetch validity + validity.FetchRow(transaction, *state.child_states[0], row_id, result, result_idx); + + // Fetch child column + auto &child_vec = ArrayVector::GetEntry(result); + auto array_size = ArrayType::GetSize(type); + child_column->FetchRow(transaction, *state.child_states[1], row_id * array_size, child_vec, result_idx); +} + +void ArrayColumnData::CommitDropColumn() { + validity.CommitDropColumn(); + child_column->CommitDropColumn(); +} + +struct ArrayColumnCheckpointState : public ColumnCheckpointState { + ArrayColumnCheckpointState(RowGroup &row_group, ColumnData &column_data, PartialBlockManager &partial_block_manager) + : ColumnCheckpointState(row_group, column_data, partial_block_manager) { + global_stats = ArrayStats::CreateEmpty(column_data.type).ToUnique(); + } + + unique_ptr validity_state; + unique_ptr child_state; + +public: + unique_ptr GetStatistics() override { + auto stats = global_stats->Copy(); + ArrayStats::SetChildStats(stats, child_state->GetStatistics()); + return stats.ToUnique(); + } + + void WriteDataPointers(RowGroupWriter &writer, Serializer &serializer) override { + serializer.WriteObject(101, "validity", + [&](Serializer &serializer) { validity_state->WriteDataPointers(writer, serializer); }); + serializer.WriteObject(102, "child_column", + [&](Serializer &serializer) { child_state->WriteDataPointers(writer, serializer); }); + } +}; + +unique_ptr ArrayColumnData::CreateCheckpointState(RowGroup &row_group, + PartialBlockManager &partial_block_manager) { + return make_uniq(row_group, *this, partial_block_manager); +} + +unique_ptr ArrayColumnData::Checkpoint(RowGroup &row_group, + PartialBlockManager &partial_block_manager, + ColumnCheckpointInfo &checkpoint_info) { + + auto checkpoint_state = make_uniq(row_group, *this, partial_block_manager); + checkpoint_state->validity_state = validity.Checkpoint(row_group, partial_block_manager, checkpoint_info); + checkpoint_state->child_state = child_column->Checkpoint(row_group, partial_block_manager, checkpoint_info); + return std::move(checkpoint_state); +} + +void ArrayColumnData::DeserializeColumn(Deserializer &deserializer) { + deserializer.ReadObject(101, "validity", [&](Deserializer &source) { validity.DeserializeColumn(source); }); + deserializer.ReadObject(102, "child_column", + [&](Deserializer &source) { child_column->DeserializeColumn(source); }); + this->count = validity.count; +} + +void ArrayColumnData::GetColumnSegmentInfo(idx_t row_group_index, vector col_path, + vector &result) { + col_path.push_back(0); + validity.GetColumnSegmentInfo(row_group_index, col_path, result); + col_path.back() = 1; + child_column->GetColumnSegmentInfo(row_group_index, col_path, result); +} + +void ArrayColumnData::Verify(RowGroup &parent) { +#ifdef DEBUG + ColumnData::Verify(parent); + validity.Verify(parent); + child_column->Verify(parent); +#endif +} + +} // namespace duckdb diff --git a/src/duckdb/src/storage/table/column_data.cpp b/src/duckdb/src/storage/table/column_data.cpp index d6a1bd938..e61af4d83 100644 --- a/src/duckdb/src/storage/table/column_data.cpp +++ b/src/duckdb/src/storage/table/column_data.cpp @@ -9,7 +9,7 @@ #include "duckdb/storage/table/column_data_checkpointer.hpp" #include "duckdb/storage/table/list_column_data.hpp" #include "duckdb/storage/table/standard_column_data.hpp" - +#include "duckdb/storage/table/array_column_data.hpp" #include "duckdb/storage/table/struct_column_data.hpp" #include "duckdb/storage/table/update_segment.hpp" #include "duckdb/storage/table_storage_info.hpp" @@ -544,8 +544,8 @@ void ColumnData::Verify(RowGroup &parent) { #ifdef DEBUG D_ASSERT(this->start == parent.start); data.Verify(); - if (type.InternalType() == PhysicalType::STRUCT) { - // structs don't have segments + if (type.InternalType() == PhysicalType::STRUCT || type.InternalType() == PhysicalType::ARRAY) { + // structs and fixed size lists don't have segments D_ASSERT(!data.GetRootSegment()); return; } @@ -570,6 +570,8 @@ static RET CreateColumnInternal(BlockManager &block_manager, DataTableInfo &info return OP::template Create(block_manager, info, column_index, start_row, type, parent); } else if (type.InternalType() == PhysicalType::LIST) { return OP::template Create(block_manager, info, column_index, start_row, type, parent); + } else if (type.InternalType() == PhysicalType::ARRAY) { + return OP::template Create(block_manager, info, column_index, start_row, type, parent); } else if (type.id() == LogicalTypeId::VALIDITY) { return OP::template Create(block_manager, info, column_index, start_row, *parent); } diff --git a/src/duckdb/src/storage/table/list_column_data.cpp b/src/duckdb/src/storage/table/list_column_data.cpp index 36f6da70c..2135f863c 100644 --- a/src/duckdb/src/storage/table/list_column_data.cpp +++ b/src/duckdb/src/storage/table/list_column_data.cpp @@ -113,6 +113,7 @@ idx_t ListColumnData::ScanCount(ColumnScanState &state, Vector &result, idx_t co if (child_scan_count > 0) { auto &child_entry = ListVector::GetEntry(result); if (child_entry.GetType().InternalType() != PhysicalType::STRUCT && + child_entry.GetType().InternalType() != PhysicalType::ARRAY && state.child_states[1].row_index + child_scan_count > child_column->start + child_column->GetMaxEntry()) { throw InternalException("ListColumnData::ScanCount - internal list scan offset is out of range"); } diff --git a/src/duckdb/src/storage/table/row_group.cpp b/src/duckdb/src/storage/table/row_group.cpp index eafe98043..73fe4e6a3 100644 --- a/src/duckdb/src/storage/table/row_group.cpp +++ b/src/duckdb/src/storage/table/row_group.cpp @@ -141,6 +141,10 @@ void ColumnScanState::Initialize(const LogicalType &type) { // validity + list child child_states.resize(2); child_states[1].Initialize(ListType::GetChildType(type)); + } else if (type.InternalType() == PhysicalType::ARRAY) { + // validity + array child + child_states.resize(2); + child_states[1].Initialize(ArrayType::GetChildType(type)); } else { // validity child_states.resize(1); diff --git a/src/duckdb/src/storage/table/row_version_manager.cpp b/src/duckdb/src/storage/table/row_version_manager.cpp index 945d0a3b1..9fa7c46cb 100644 --- a/src/duckdb/src/storage/table/row_version_manager.cpp +++ b/src/duckdb/src/storage/table/row_version_manager.cpp @@ -110,9 +110,6 @@ void RowVersionManager::AppendVersionInfo(TransactionData transaction, idx_t cou } void RowVersionManager::CommitAppend(transaction_t commit_id, idx_t row_group_start, idx_t count) { - if (count == 0) { - return; - } idx_t row_group_end = row_group_start + count; lock_guard lock(version_lock); @@ -122,8 +119,9 @@ void RowVersionManager::CommitAppend(transaction_t commit_id, idx_t row_group_st idx_t vstart = vector_idx == start_vector_idx ? row_group_start - start_vector_idx * STANDARD_VECTOR_SIZE : 0; idx_t vend = vector_idx == end_vector_idx ? row_group_end - end_vector_idx * STANDARD_VECTOR_SIZE : STANDARD_VECTOR_SIZE; - auto &info = *vector_info[vector_idx]; - info.CommitAppend(commit_id, vstart, vend); + + auto info = vector_info[vector_idx].get(); + info->CommitAppend(commit_id, vstart, vend); } } diff --git a/src/duckdb/third_party/libpg_query/include/nodes/parsenodes.hpp b/src/duckdb/third_party/libpg_query/include/nodes/parsenodes.hpp index 3ecf1b8bb..ee2d7dc88 100644 --- a/src/duckdb/third_party/libpg_query/include/nodes/parsenodes.hpp +++ b/src/duckdb/third_party/libpg_query/include/nodes/parsenodes.hpp @@ -438,27 +438,37 @@ typedef struct PGWindowDef { * which were defaulted; the correct behavioral bits must be set either way. * The START_foo and END_foo options must come in pairs of adjacent bits for * the convenience of gram.y, even though some of them are useless/invalid. - * We will need more bits (and fields) to cover the full SQL:2008 option set. - */ -#define FRAMEOPTION_NONDEFAULT 0x00001 /* any specified? */ -#define FRAMEOPTION_RANGE 0x00002 /* RANGE behavior */ -#define FRAMEOPTION_ROWS 0x00004 /* ROWS behavior */ -#define FRAMEOPTION_BETWEEN 0x00008 /* BETWEEN given? */ -#define FRAMEOPTION_START_UNBOUNDED_PRECEDING 0x00010 /* start is U. P. */ -#define FRAMEOPTION_END_UNBOUNDED_PRECEDING 0x00020 /* (disallowed) */ -#define FRAMEOPTION_START_UNBOUNDED_FOLLOWING 0x00040 /* (disallowed) */ -#define FRAMEOPTION_END_UNBOUNDED_FOLLOWING 0x00080 /* end is U. F. */ -#define FRAMEOPTION_START_CURRENT_ROW 0x00100 /* start is C. R. */ -#define FRAMEOPTION_END_CURRENT_ROW 0x00200 /* end is C. R. */ -#define FRAMEOPTION_START_VALUE_PRECEDING 0x00400 /* start is V. P. */ -#define FRAMEOPTION_END_VALUE_PRECEDING 0x00800 /* end is V. P. */ -#define FRAMEOPTION_START_VALUE_FOLLOWING 0x01000 /* start is V. F. */ -#define FRAMEOPTION_END_VALUE_FOLLOWING 0x02000 /* end is V. F. */ - -#define FRAMEOPTION_START_VALUE (FRAMEOPTION_START_VALUE_PRECEDING | FRAMEOPTION_START_VALUE_FOLLOWING) -#define FRAMEOPTION_END_VALUE (FRAMEOPTION_END_VALUE_PRECEDING | FRAMEOPTION_END_VALUE_FOLLOWING) - -#define FRAMEOPTION_DEFAULTS (FRAMEOPTION_RANGE | FRAMEOPTION_START_UNBOUNDED_PRECEDING | FRAMEOPTION_END_CURRENT_ROW) + */ +#define FRAMEOPTION_NONDEFAULT 0x00001 /* any specified? */ +#define FRAMEOPTION_RANGE 0x00002 /* RANGE behavior */ +#define FRAMEOPTION_ROWS 0x00004 /* ROWS behavior */ +#define FRAMEOPTION_GROUPS 0x00008 /* GROUPS behavior */ +#define FRAMEOPTION_BETWEEN 0x00010 /* BETWEEN given? */ +#define FRAMEOPTION_START_UNBOUNDED_PRECEDING 0x00020 /* start is U. P. */ +#define FRAMEOPTION_END_UNBOUNDED_PRECEDING 0x00040 /* (disallowed) */ +#define FRAMEOPTION_START_UNBOUNDED_FOLLOWING 0x00080 /* (disallowed) */ +#define FRAMEOPTION_END_UNBOUNDED_FOLLOWING 0x00100 /* end is U. F. */ +#define FRAMEOPTION_START_CURRENT_ROW 0x00200 /* start is C. R. */ +#define FRAMEOPTION_END_CURRENT_ROW 0x00400 /* end is C. R. */ +#define FRAMEOPTION_START_OFFSET_PRECEDING 0x00800 /* start is O. P. */ +#define FRAMEOPTION_END_OFFSET_PRECEDING 0x01000 /* end is O. P. */ +#define FRAMEOPTION_START_OFFSET_FOLLOWING 0x02000 /* start is O. F. */ +#define FRAMEOPTION_END_OFFSET_FOLLOWING 0x04000 /* end is O. F. */ +#define FRAMEOPTION_EXCLUDE_CURRENT_ROW 0x08000 /* omit C.R. */ +#define FRAMEOPTION_EXCLUDE_GROUP 0x10000 /* omit C.R. & peers */ +#define FRAMEOPTION_EXCLUDE_TIES 0x20000 /* omit C.R.'s peers */ + +#define FRAMEOPTION_START_OFFSET \ + (FRAMEOPTION_START_OFFSET_PRECEDING | FRAMEOPTION_START_OFFSET_FOLLOWING) +#define FRAMEOPTION_END_OFFSET \ + (FRAMEOPTION_END_OFFSET_PRECEDING | FRAMEOPTION_END_OFFSET_FOLLOWING) +#define FRAMEOPTION_EXCLUSION \ + (FRAMEOPTION_EXCLUDE_CURRENT_ROW | FRAMEOPTION_EXCLUDE_GROUP | \ + FRAMEOPTION_EXCLUDE_TIES) + +#define FRAMEOPTION_DEFAULTS \ + (FRAMEOPTION_RANGE | FRAMEOPTION_START_UNBOUNDED_PRECEDING | \ + FRAMEOPTION_END_CURRENT_ROW) /* * PGRangeSubselect - subquery appearing in a FROM clause diff --git a/src/duckdb/third_party/libpg_query/include/parser/gram.hpp b/src/duckdb/third_party/libpg_query/include/parser/gram.hpp index 8412c59fb..11fe54622 100644 --- a/src/duckdb/third_party/libpg_query/include/parser/gram.hpp +++ b/src/duckdb/third_party/libpg_query/include/parser/gram.hpp @@ -215,316 +215,319 @@ GROUP_P = 431, GROUPING = 432, GROUPING_ID = 433, - HANDLER = 434, - HAVING = 435, - HEADER_P = 436, - HOLD = 437, - HOUR_P = 438, - HOURS_P = 439, - IDENTITY_P = 440, - IF_P = 441, - IGNORE_P = 442, - ILIKE = 443, - IMMEDIATE = 444, - IMMUTABLE = 445, - IMPLICIT_P = 446, - IMPORT_P = 447, - IN_P = 448, - INCLUDE_P = 449, - INCLUDING = 450, - INCREMENT = 451, - INDEX = 452, - INDEXES = 453, - INHERIT = 454, - INHERITS = 455, - INITIALLY = 456, - INLINE_P = 457, - INNER_P = 458, - INOUT = 459, - INPUT_P = 460, - INSENSITIVE = 461, - INSERT = 462, - INSTALL = 463, - INSTEAD = 464, - INT_P = 465, - INTEGER = 466, - INTERSECT = 467, - INTERVAL = 468, - INTO = 469, - INVOKER = 470, - IS = 471, - ISNULL = 472, - ISOLATION = 473, - JOIN = 474, - JSON = 475, - KEY = 476, - LABEL = 477, - LANGUAGE = 478, - LARGE_P = 479, - LAST_P = 480, - LATERAL_P = 481, - LEADING = 482, - LEAKPROOF = 483, - LEFT = 484, - LEVEL = 485, - LIKE = 486, - LIMIT = 487, - LISTEN = 488, - LOAD = 489, - LOCAL = 490, - LOCATION = 491, - LOCK_P = 492, - LOCKED = 493, - LOGGED = 494, - MACRO = 495, - MAP = 496, - MAPPING = 497, - MATCH = 498, - MATERIALIZED = 499, - MAXVALUE = 500, - METHOD = 501, - MICROSECOND_P = 502, - MICROSECONDS_P = 503, - MILLISECOND_P = 504, - MILLISECONDS_P = 505, - MINUTE_P = 506, - MINUTES_P = 507, - MINVALUE = 508, - MODE = 509, - MONTH_P = 510, - MONTHS_P = 511, - MOVE = 512, - NAME_P = 513, - NAMES = 514, - NATIONAL = 515, - NATURAL = 516, - NCHAR = 517, - NEW = 518, - NEXT = 519, - NO = 520, - NONE = 521, - NOT = 522, - NOTHING = 523, - NOTIFY = 524, - NOTNULL = 525, - NOWAIT = 526, - NULL_P = 527, - NULLIF = 528, - NULLS_P = 529, - NUMERIC = 530, - OBJECT_P = 531, - OF = 532, - OFF = 533, - OFFSET = 534, - OIDS = 535, - OLD = 536, - ON = 537, - ONLY = 538, - OPERATOR = 539, - OPTION = 540, - OPTIONS = 541, - OR = 542, - ORDER = 543, - ORDINALITY = 544, - OUT_P = 545, - OUTER_P = 546, - OVER = 547, - OVERLAPS = 548, - OVERLAY = 549, - OVERRIDING = 550, - OWNED = 551, - OWNER = 552, - PARALLEL = 553, - PARSER = 554, - PARTIAL = 555, - PARTITION = 556, - PASSING = 557, - PASSWORD = 558, - PERCENT = 559, - PIVOT = 560, - PIVOT_LONGER = 561, - PIVOT_WIDER = 562, - PLACING = 563, - PLANS = 564, - POLICY = 565, - POSITION = 566, - POSITIONAL = 567, - PRAGMA_P = 568, - PRECEDING = 569, - PRECISION = 570, - PREPARE = 571, - PREPARED = 572, - PRESERVE = 573, - PRIMARY = 574, - PRIOR = 575, - PRIVILEGES = 576, - PROCEDURAL = 577, - PROCEDURE = 578, - PROGRAM = 579, - PUBLICATION = 580, - QUALIFY = 581, - QUOTE = 582, - RANGE = 583, - READ_P = 584, - REAL = 585, - REASSIGN = 586, - RECHECK = 587, - RECURSIVE = 588, - REF = 589, - REFERENCES = 590, - REFERENCING = 591, - REFRESH = 592, - REINDEX = 593, - RELATIVE_P = 594, - RELEASE = 595, - RENAME = 596, - REPEATABLE = 597, - REPLACE = 598, - REPLICA = 599, - RESET = 600, - RESPECT_P = 601, - RESTART = 602, - RESTRICT = 603, - RETURNING = 604, - RETURNS = 605, - REVOKE = 606, - RIGHT = 607, - ROLE = 608, - ROLLBACK = 609, - ROLLUP = 610, - ROW = 611, - ROWS = 612, - RULE = 613, - SAMPLE = 614, - SAVEPOINT = 615, - SCHEMA = 616, - SCHEMAS = 617, - SCROLL = 618, - SEARCH = 619, - SECOND_P = 620, - SECONDS_P = 621, - SECURITY = 622, - SELECT = 623, - SEMI = 624, - SEQUENCE = 625, - SEQUENCES = 626, - SERIALIZABLE = 627, - SERVER = 628, - SESSION = 629, - SET = 630, - SETOF = 631, - SETS = 632, - SHARE = 633, - SHOW = 634, - SIMILAR = 635, - SIMPLE = 636, - SKIP = 637, - SMALLINT = 638, - SNAPSHOT = 639, - SOME = 640, - SQL_P = 641, - STABLE = 642, - STANDALONE_P = 643, - START = 644, - STATEMENT = 645, - STATISTICS = 646, - STDIN = 647, - STDOUT = 648, - STORAGE = 649, - STORED = 650, - STRICT_P = 651, - STRIP_P = 652, - STRUCT = 653, - SUBSCRIPTION = 654, - SUBSTRING = 655, - SUMMARIZE = 656, - SYMMETRIC = 657, - SYSID = 658, - SYSTEM_P = 659, - TABLE = 660, - TABLES = 661, - TABLESAMPLE = 662, - TABLESPACE = 663, - TEMP = 664, - TEMPLATE = 665, - TEMPORARY = 666, - TEXT_P = 667, - THEN = 668, - TIME = 669, - TIMESTAMP = 670, - TO = 671, - TRAILING = 672, - TRANSACTION = 673, - TRANSFORM = 674, - TREAT = 675, - TRIGGER = 676, - TRIM = 677, - TRUE_P = 678, - TRUNCATE = 679, - TRUSTED = 680, - TRY_CAST = 681, - TYPE_P = 682, - TYPES_P = 683, - UNBOUNDED = 684, - UNCOMMITTED = 685, - UNENCRYPTED = 686, - UNION = 687, - UNIQUE = 688, - UNKNOWN = 689, - UNLISTEN = 690, - UNLOGGED = 691, - UNPIVOT = 692, - UNTIL = 693, - UPDATE = 694, - USE_P = 695, - USER = 696, - USING = 697, - VACUUM = 698, - VALID = 699, - VALIDATE = 700, - VALIDATOR = 701, - VALUE_P = 702, - VALUES = 703, - VARCHAR = 704, - VARIADIC = 705, - VARYING = 706, - VERBOSE = 707, - VERSION_P = 708, - VIEW = 709, - VIEWS = 710, - VIRTUAL = 711, - VOLATILE = 712, - WHEN = 713, - WHERE = 714, - WHITESPACE_P = 715, - WINDOW = 716, - WITH = 717, - WITHIN = 718, - WITHOUT = 719, - WORK = 720, - WRAPPER = 721, - WRITE_P = 722, - XML_P = 723, - XMLATTRIBUTES = 724, - XMLCONCAT = 725, - XMLELEMENT = 726, - XMLEXISTS = 727, - XMLFOREST = 728, - XMLNAMESPACES = 729, - XMLPARSE = 730, - XMLPI = 731, - XMLROOT = 732, - XMLSERIALIZE = 733, - XMLTABLE = 734, - YEAR_P = 735, - YEARS_P = 736, - YES_P = 737, - ZONE = 738, - NOT_LA = 739, - NULLS_LA = 740, - WITH_LA = 741, - POSTFIXOP = 742, - UMINUS = 743 + GROUPS = 434, + HANDLER = 435, + HAVING = 436, + HEADER_P = 437, + HOLD = 438, + HOUR_P = 439, + HOURS_P = 440, + IDENTITY_P = 441, + IF_P = 442, + IGNORE_P = 443, + ILIKE = 444, + IMMEDIATE = 445, + IMMUTABLE = 446, + IMPLICIT_P = 447, + IMPORT_P = 448, + IN_P = 449, + INCLUDE_P = 450, + INCLUDING = 451, + INCREMENT = 452, + INDEX = 453, + INDEXES = 454, + INHERIT = 455, + INHERITS = 456, + INITIALLY = 457, + INLINE_P = 458, + INNER_P = 459, + INOUT = 460, + INPUT_P = 461, + INSENSITIVE = 462, + INSERT = 463, + INSTALL = 464, + INSTEAD = 465, + INT_P = 466, + INTEGER = 467, + INTERSECT = 468, + INTERVAL = 469, + INTO = 470, + INVOKER = 471, + IS = 472, + ISNULL = 473, + ISOLATION = 474, + JOIN = 475, + JSON = 476, + KEY = 477, + LABEL = 478, + LANGUAGE = 479, + LARGE_P = 480, + LAST_P = 481, + LATERAL_P = 482, + LEADING = 483, + LEAKPROOF = 484, + LEFT = 485, + LEVEL = 486, + LIKE = 487, + LIMIT = 488, + LISTEN = 489, + LOAD = 490, + LOCAL = 491, + LOCATION = 492, + LOCK_P = 493, + LOCKED = 494, + LOGGED = 495, + MACRO = 496, + MAP = 497, + MAPPING = 498, + MATCH = 499, + MATERIALIZED = 500, + MAXVALUE = 501, + METHOD = 502, + MICROSECOND_P = 503, + MICROSECONDS_P = 504, + MILLISECOND_P = 505, + MILLISECONDS_P = 506, + MINUTE_P = 507, + MINUTES_P = 508, + MINVALUE = 509, + MODE = 510, + MONTH_P = 511, + MONTHS_P = 512, + MOVE = 513, + NAME_P = 514, + NAMES = 515, + NATIONAL = 516, + NATURAL = 517, + NCHAR = 518, + NEW = 519, + NEXT = 520, + NO = 521, + NONE = 522, + NOT = 523, + NOTHING = 524, + NOTIFY = 525, + NOTNULL = 526, + NOWAIT = 527, + NULL_P = 528, + NULLIF = 529, + NULLS_P = 530, + NUMERIC = 531, + OBJECT_P = 532, + OF = 533, + OFF = 534, + OFFSET = 535, + OIDS = 536, + OLD = 537, + ON = 538, + ONLY = 539, + OPERATOR = 540, + OPTION = 541, + OPTIONS = 542, + OR = 543, + ORDER = 544, + ORDINALITY = 545, + OTHERS = 546, + OUT_P = 547, + OUTER_P = 548, + OVER = 549, + OVERLAPS = 550, + OVERLAY = 551, + OVERRIDING = 552, + OWNED = 553, + OWNER = 554, + PARALLEL = 555, + PARSER = 556, + PARTIAL = 557, + PARTITION = 558, + PASSING = 559, + PASSWORD = 560, + PERCENT = 561, + PIVOT = 562, + PIVOT_LONGER = 563, + PIVOT_WIDER = 564, + PLACING = 565, + PLANS = 566, + POLICY = 567, + POSITION = 568, + POSITIONAL = 569, + PRAGMA_P = 570, + PRECEDING = 571, + PRECISION = 572, + PREPARE = 573, + PREPARED = 574, + PRESERVE = 575, + PRIMARY = 576, + PRIOR = 577, + PRIVILEGES = 578, + PROCEDURAL = 579, + PROCEDURE = 580, + PROGRAM = 581, + PUBLICATION = 582, + QUALIFY = 583, + QUOTE = 584, + RANGE = 585, + READ_P = 586, + REAL = 587, + REASSIGN = 588, + RECHECK = 589, + RECURSIVE = 590, + REF = 591, + REFERENCES = 592, + REFERENCING = 593, + REFRESH = 594, + REINDEX = 595, + RELATIVE_P = 596, + RELEASE = 597, + RENAME = 598, + REPEATABLE = 599, + REPLACE = 600, + REPLICA = 601, + RESET = 602, + RESPECT_P = 603, + RESTART = 604, + RESTRICT = 605, + RETURNING = 606, + RETURNS = 607, + REVOKE = 608, + RIGHT = 609, + ROLE = 610, + ROLLBACK = 611, + ROLLUP = 612, + ROW = 613, + ROWS = 614, + RULE = 615, + SAMPLE = 616, + SAVEPOINT = 617, + SCHEMA = 618, + SCHEMAS = 619, + SCROLL = 620, + SEARCH = 621, + SECOND_P = 622, + SECONDS_P = 623, + SECURITY = 624, + SELECT = 625, + SEMI = 626, + SEQUENCE = 627, + SEQUENCES = 628, + SERIALIZABLE = 629, + SERVER = 630, + SESSION = 631, + SET = 632, + SETOF = 633, + SETS = 634, + SHARE = 635, + SHOW = 636, + SIMILAR = 637, + SIMPLE = 638, + SKIP = 639, + SMALLINT = 640, + SNAPSHOT = 641, + SOME = 642, + SQL_P = 643, + STABLE = 644, + STANDALONE_P = 645, + START = 646, + STATEMENT = 647, + STATISTICS = 648, + STDIN = 649, + STDOUT = 650, + STORAGE = 651, + STORED = 652, + STRICT_P = 653, + STRIP_P = 654, + STRUCT = 655, + SUBSCRIPTION = 656, + SUBSTRING = 657, + SUMMARIZE = 658, + SYMMETRIC = 659, + SYSID = 660, + SYSTEM_P = 661, + TABLE = 662, + TABLES = 663, + TABLESAMPLE = 664, + TABLESPACE = 665, + TEMP = 666, + TEMPLATE = 667, + TEMPORARY = 668, + TEXT_P = 669, + THEN = 670, + TIES = 671, + TIME = 672, + TIMESTAMP = 673, + TO = 674, + TRAILING = 675, + TRANSACTION = 676, + TRANSFORM = 677, + TREAT = 678, + TRIGGER = 679, + TRIM = 680, + TRUE_P = 681, + TRUNCATE = 682, + TRUSTED = 683, + TRY_CAST = 684, + TYPE_P = 685, + TYPES_P = 686, + UNBOUNDED = 687, + UNCOMMITTED = 688, + UNENCRYPTED = 689, + UNION = 690, + UNIQUE = 691, + UNKNOWN = 692, + UNLISTEN = 693, + UNLOGGED = 694, + UNPIVOT = 695, + UNTIL = 696, + UPDATE = 697, + USE_P = 698, + USER = 699, + USING = 700, + VACUUM = 701, + VALID = 702, + VALIDATE = 703, + VALIDATOR = 704, + VALUE_P = 705, + VALUES = 706, + VARCHAR = 707, + VARIADIC = 708, + VARYING = 709, + VERBOSE = 710, + VERSION_P = 711, + VIEW = 712, + VIEWS = 713, + VIRTUAL = 714, + VOLATILE = 715, + WHEN = 716, + WHERE = 717, + WHITESPACE_P = 718, + WINDOW = 719, + WITH = 720, + WITHIN = 721, + WITHOUT = 722, + WORK = 723, + WRAPPER = 724, + WRITE_P = 725, + XML_P = 726, + XMLATTRIBUTES = 727, + XMLCONCAT = 728, + XMLELEMENT = 729, + XMLEXISTS = 730, + XMLFOREST = 731, + XMLNAMESPACES = 732, + XMLPARSE = 733, + XMLPI = 734, + XMLROOT = 735, + XMLSERIALIZE = 736, + XMLTABLE = 737, + YEAR_P = 738, + YEARS_P = 739, + YES_P = 740, + ZONE = 741, + NOT_LA = 742, + NULLS_LA = 743, + WITH_LA = 744, + POSTFIXOP = 745, + UMINUS = 746 }; #endif /* Tokens. */ @@ -704,316 +707,319 @@ #define GROUP_P 431 #define GROUPING 432 #define GROUPING_ID 433 -#define HANDLER 434 -#define HAVING 435 -#define HEADER_P 436 -#define HOLD 437 -#define HOUR_P 438 -#define HOURS_P 439 -#define IDENTITY_P 440 -#define IF_P 441 -#define IGNORE_P 442 -#define ILIKE 443 -#define IMMEDIATE 444 -#define IMMUTABLE 445 -#define IMPLICIT_P 446 -#define IMPORT_P 447 -#define IN_P 448 -#define INCLUDE_P 449 -#define INCLUDING 450 -#define INCREMENT 451 -#define INDEX 452 -#define INDEXES 453 -#define INHERIT 454 -#define INHERITS 455 -#define INITIALLY 456 -#define INLINE_P 457 -#define INNER_P 458 -#define INOUT 459 -#define INPUT_P 460 -#define INSENSITIVE 461 -#define INSERT 462 -#define INSTALL 463 -#define INSTEAD 464 -#define INT_P 465 -#define INTEGER 466 -#define INTERSECT 467 -#define INTERVAL 468 -#define INTO 469 -#define INVOKER 470 -#define IS 471 -#define ISNULL 472 -#define ISOLATION 473 -#define JOIN 474 -#define JSON 475 -#define KEY 476 -#define LABEL 477 -#define LANGUAGE 478 -#define LARGE_P 479 -#define LAST_P 480 -#define LATERAL_P 481 -#define LEADING 482 -#define LEAKPROOF 483 -#define LEFT 484 -#define LEVEL 485 -#define LIKE 486 -#define LIMIT 487 -#define LISTEN 488 -#define LOAD 489 -#define LOCAL 490 -#define LOCATION 491 -#define LOCK_P 492 -#define LOCKED 493 -#define LOGGED 494 -#define MACRO 495 -#define MAP 496 -#define MAPPING 497 -#define MATCH 498 -#define MATERIALIZED 499 -#define MAXVALUE 500 -#define METHOD 501 -#define MICROSECOND_P 502 -#define MICROSECONDS_P 503 -#define MILLISECOND_P 504 -#define MILLISECONDS_P 505 -#define MINUTE_P 506 -#define MINUTES_P 507 -#define MINVALUE 508 -#define MODE 509 -#define MONTH_P 510 -#define MONTHS_P 511 -#define MOVE 512 -#define NAME_P 513 -#define NAMES 514 -#define NATIONAL 515 -#define NATURAL 516 -#define NCHAR 517 -#define NEW 518 -#define NEXT 519 -#define NO 520 -#define NONE 521 -#define NOT 522 -#define NOTHING 523 -#define NOTIFY 524 -#define NOTNULL 525 -#define NOWAIT 526 -#define NULL_P 527 -#define NULLIF 528 -#define NULLS_P 529 -#define NUMERIC 530 -#define OBJECT_P 531 -#define OF 532 -#define OFF 533 -#define OFFSET 534 -#define OIDS 535 -#define OLD 536 -#define ON 537 -#define ONLY 538 -#define OPERATOR 539 -#define OPTION 540 -#define OPTIONS 541 -#define OR 542 -#define ORDER 543 -#define ORDINALITY 544 -#define OUT_P 545 -#define OUTER_P 546 -#define OVER 547 -#define OVERLAPS 548 -#define OVERLAY 549 -#define OVERRIDING 550 -#define OWNED 551 -#define OWNER 552 -#define PARALLEL 553 -#define PARSER 554 -#define PARTIAL 555 -#define PARTITION 556 -#define PASSING 557 -#define PASSWORD 558 -#define PERCENT 559 -#define PIVOT 560 -#define PIVOT_LONGER 561 -#define PIVOT_WIDER 562 -#define PLACING 563 -#define PLANS 564 -#define POLICY 565 -#define POSITION 566 -#define POSITIONAL 567 -#define PRAGMA_P 568 -#define PRECEDING 569 -#define PRECISION 570 -#define PREPARE 571 -#define PREPARED 572 -#define PRESERVE 573 -#define PRIMARY 574 -#define PRIOR 575 -#define PRIVILEGES 576 -#define PROCEDURAL 577 -#define PROCEDURE 578 -#define PROGRAM 579 -#define PUBLICATION 580 -#define QUALIFY 581 -#define QUOTE 582 -#define RANGE 583 -#define READ_P 584 -#define REAL 585 -#define REASSIGN 586 -#define RECHECK 587 -#define RECURSIVE 588 -#define REF 589 -#define REFERENCES 590 -#define REFERENCING 591 -#define REFRESH 592 -#define REINDEX 593 -#define RELATIVE_P 594 -#define RELEASE 595 -#define RENAME 596 -#define REPEATABLE 597 -#define REPLACE 598 -#define REPLICA 599 -#define RESET 600 -#define RESPECT_P 601 -#define RESTART 602 -#define RESTRICT 603 -#define RETURNING 604 -#define RETURNS 605 -#define REVOKE 606 -#define RIGHT 607 -#define ROLE 608 -#define ROLLBACK 609 -#define ROLLUP 610 -#define ROW 611 -#define ROWS 612 -#define RULE 613 -#define SAMPLE 614 -#define SAVEPOINT 615 -#define SCHEMA 616 -#define SCHEMAS 617 -#define SCROLL 618 -#define SEARCH 619 -#define SECOND_P 620 -#define SECONDS_P 621 -#define SECURITY 622 -#define SELECT 623 -#define SEMI 624 -#define SEQUENCE 625 -#define SEQUENCES 626 -#define SERIALIZABLE 627 -#define SERVER 628 -#define SESSION 629 -#define SET 630 -#define SETOF 631 -#define SETS 632 -#define SHARE 633 -#define SHOW 634 -#define SIMILAR 635 -#define SIMPLE 636 -#define SKIP 637 -#define SMALLINT 638 -#define SNAPSHOT 639 -#define SOME 640 -#define SQL_P 641 -#define STABLE 642 -#define STANDALONE_P 643 -#define START 644 -#define STATEMENT 645 -#define STATISTICS 646 -#define STDIN 647 -#define STDOUT 648 -#define STORAGE 649 -#define STORED 650 -#define STRICT_P 651 -#define STRIP_P 652 -#define STRUCT 653 -#define SUBSCRIPTION 654 -#define SUBSTRING 655 -#define SUMMARIZE 656 -#define SYMMETRIC 657 -#define SYSID 658 -#define SYSTEM_P 659 -#define TABLE 660 -#define TABLES 661 -#define TABLESAMPLE 662 -#define TABLESPACE 663 -#define TEMP 664 -#define TEMPLATE 665 -#define TEMPORARY 666 -#define TEXT_P 667 -#define THEN 668 -#define TIME 669 -#define TIMESTAMP 670 -#define TO 671 -#define TRAILING 672 -#define TRANSACTION 673 -#define TRANSFORM 674 -#define TREAT 675 -#define TRIGGER 676 -#define TRIM 677 -#define TRUE_P 678 -#define TRUNCATE 679 -#define TRUSTED 680 -#define TRY_CAST 681 -#define TYPE_P 682 -#define TYPES_P 683 -#define UNBOUNDED 684 -#define UNCOMMITTED 685 -#define UNENCRYPTED 686 -#define UNION 687 -#define UNIQUE 688 -#define UNKNOWN 689 -#define UNLISTEN 690 -#define UNLOGGED 691 -#define UNPIVOT 692 -#define UNTIL 693 -#define UPDATE 694 -#define USE_P 695 -#define USER 696 -#define USING 697 -#define VACUUM 698 -#define VALID 699 -#define VALIDATE 700 -#define VALIDATOR 701 -#define VALUE_P 702 -#define VALUES 703 -#define VARCHAR 704 -#define VARIADIC 705 -#define VARYING 706 -#define VERBOSE 707 -#define VERSION_P 708 -#define VIEW 709 -#define VIEWS 710 -#define VIRTUAL 711 -#define VOLATILE 712 -#define WHEN 713 -#define WHERE 714 -#define WHITESPACE_P 715 -#define WINDOW 716 -#define WITH 717 -#define WITHIN 718 -#define WITHOUT 719 -#define WORK 720 -#define WRAPPER 721 -#define WRITE_P 722 -#define XML_P 723 -#define XMLATTRIBUTES 724 -#define XMLCONCAT 725 -#define XMLELEMENT 726 -#define XMLEXISTS 727 -#define XMLFOREST 728 -#define XMLNAMESPACES 729 -#define XMLPARSE 730 -#define XMLPI 731 -#define XMLROOT 732 -#define XMLSERIALIZE 733 -#define XMLTABLE 734 -#define YEAR_P 735 -#define YEARS_P 736 -#define YES_P 737 -#define ZONE 738 -#define NOT_LA 739 -#define NULLS_LA 740 -#define WITH_LA 741 -#define POSTFIXOP 742 -#define UMINUS 743 +#define GROUPS 434 +#define HANDLER 435 +#define HAVING 436 +#define HEADER_P 437 +#define HOLD 438 +#define HOUR_P 439 +#define HOURS_P 440 +#define IDENTITY_P 441 +#define IF_P 442 +#define IGNORE_P 443 +#define ILIKE 444 +#define IMMEDIATE 445 +#define IMMUTABLE 446 +#define IMPLICIT_P 447 +#define IMPORT_P 448 +#define IN_P 449 +#define INCLUDE_P 450 +#define INCLUDING 451 +#define INCREMENT 452 +#define INDEX 453 +#define INDEXES 454 +#define INHERIT 455 +#define INHERITS 456 +#define INITIALLY 457 +#define INLINE_P 458 +#define INNER_P 459 +#define INOUT 460 +#define INPUT_P 461 +#define INSENSITIVE 462 +#define INSERT 463 +#define INSTALL 464 +#define INSTEAD 465 +#define INT_P 466 +#define INTEGER 467 +#define INTERSECT 468 +#define INTERVAL 469 +#define INTO 470 +#define INVOKER 471 +#define IS 472 +#define ISNULL 473 +#define ISOLATION 474 +#define JOIN 475 +#define JSON 476 +#define KEY 477 +#define LABEL 478 +#define LANGUAGE 479 +#define LARGE_P 480 +#define LAST_P 481 +#define LATERAL_P 482 +#define LEADING 483 +#define LEAKPROOF 484 +#define LEFT 485 +#define LEVEL 486 +#define LIKE 487 +#define LIMIT 488 +#define LISTEN 489 +#define LOAD 490 +#define LOCAL 491 +#define LOCATION 492 +#define LOCK_P 493 +#define LOCKED 494 +#define LOGGED 495 +#define MACRO 496 +#define MAP 497 +#define MAPPING 498 +#define MATCH 499 +#define MATERIALIZED 500 +#define MAXVALUE 501 +#define METHOD 502 +#define MICROSECOND_P 503 +#define MICROSECONDS_P 504 +#define MILLISECOND_P 505 +#define MILLISECONDS_P 506 +#define MINUTE_P 507 +#define MINUTES_P 508 +#define MINVALUE 509 +#define MODE 510 +#define MONTH_P 511 +#define MONTHS_P 512 +#define MOVE 513 +#define NAME_P 514 +#define NAMES 515 +#define NATIONAL 516 +#define NATURAL 517 +#define NCHAR 518 +#define NEW 519 +#define NEXT 520 +#define NO 521 +#define NONE 522 +#define NOT 523 +#define NOTHING 524 +#define NOTIFY 525 +#define NOTNULL 526 +#define NOWAIT 527 +#define NULL_P 528 +#define NULLIF 529 +#define NULLS_P 530 +#define NUMERIC 531 +#define OBJECT_P 532 +#define OF 533 +#define OFF 534 +#define OFFSET 535 +#define OIDS 536 +#define OLD 537 +#define ON 538 +#define ONLY 539 +#define OPERATOR 540 +#define OPTION 541 +#define OPTIONS 542 +#define OR 543 +#define ORDER 544 +#define ORDINALITY 545 +#define OTHERS 546 +#define OUT_P 547 +#define OUTER_P 548 +#define OVER 549 +#define OVERLAPS 550 +#define OVERLAY 551 +#define OVERRIDING 552 +#define OWNED 553 +#define OWNER 554 +#define PARALLEL 555 +#define PARSER 556 +#define PARTIAL 557 +#define PARTITION 558 +#define PASSING 559 +#define PASSWORD 560 +#define PERCENT 561 +#define PIVOT 562 +#define PIVOT_LONGER 563 +#define PIVOT_WIDER 564 +#define PLACING 565 +#define PLANS 566 +#define POLICY 567 +#define POSITION 568 +#define POSITIONAL 569 +#define PRAGMA_P 570 +#define PRECEDING 571 +#define PRECISION 572 +#define PREPARE 573 +#define PREPARED 574 +#define PRESERVE 575 +#define PRIMARY 576 +#define PRIOR 577 +#define PRIVILEGES 578 +#define PROCEDURAL 579 +#define PROCEDURE 580 +#define PROGRAM 581 +#define PUBLICATION 582 +#define QUALIFY 583 +#define QUOTE 584 +#define RANGE 585 +#define READ_P 586 +#define REAL 587 +#define REASSIGN 588 +#define RECHECK 589 +#define RECURSIVE 590 +#define REF 591 +#define REFERENCES 592 +#define REFERENCING 593 +#define REFRESH 594 +#define REINDEX 595 +#define RELATIVE_P 596 +#define RELEASE 597 +#define RENAME 598 +#define REPEATABLE 599 +#define REPLACE 600 +#define REPLICA 601 +#define RESET 602 +#define RESPECT_P 603 +#define RESTART 604 +#define RESTRICT 605 +#define RETURNING 606 +#define RETURNS 607 +#define REVOKE 608 +#define RIGHT 609 +#define ROLE 610 +#define ROLLBACK 611 +#define ROLLUP 612 +#define ROW 613 +#define ROWS 614 +#define RULE 615 +#define SAMPLE 616 +#define SAVEPOINT 617 +#define SCHEMA 618 +#define SCHEMAS 619 +#define SCROLL 620 +#define SEARCH 621 +#define SECOND_P 622 +#define SECONDS_P 623 +#define SECURITY 624 +#define SELECT 625 +#define SEMI 626 +#define SEQUENCE 627 +#define SEQUENCES 628 +#define SERIALIZABLE 629 +#define SERVER 630 +#define SESSION 631 +#define SET 632 +#define SETOF 633 +#define SETS 634 +#define SHARE 635 +#define SHOW 636 +#define SIMILAR 637 +#define SIMPLE 638 +#define SKIP 639 +#define SMALLINT 640 +#define SNAPSHOT 641 +#define SOME 642 +#define SQL_P 643 +#define STABLE 644 +#define STANDALONE_P 645 +#define START 646 +#define STATEMENT 647 +#define STATISTICS 648 +#define STDIN 649 +#define STDOUT 650 +#define STORAGE 651 +#define STORED 652 +#define STRICT_P 653 +#define STRIP_P 654 +#define STRUCT 655 +#define SUBSCRIPTION 656 +#define SUBSTRING 657 +#define SUMMARIZE 658 +#define SYMMETRIC 659 +#define SYSID 660 +#define SYSTEM_P 661 +#define TABLE 662 +#define TABLES 663 +#define TABLESAMPLE 664 +#define TABLESPACE 665 +#define TEMP 666 +#define TEMPLATE 667 +#define TEMPORARY 668 +#define TEXT_P 669 +#define THEN 670 +#define TIES 671 +#define TIME 672 +#define TIMESTAMP 673 +#define TO 674 +#define TRAILING 675 +#define TRANSACTION 676 +#define TRANSFORM 677 +#define TREAT 678 +#define TRIGGER 679 +#define TRIM 680 +#define TRUE_P 681 +#define TRUNCATE 682 +#define TRUSTED 683 +#define TRY_CAST 684 +#define TYPE_P 685 +#define TYPES_P 686 +#define UNBOUNDED 687 +#define UNCOMMITTED 688 +#define UNENCRYPTED 689 +#define UNION 690 +#define UNIQUE 691 +#define UNKNOWN 692 +#define UNLISTEN 693 +#define UNLOGGED 694 +#define UNPIVOT 695 +#define UNTIL 696 +#define UPDATE 697 +#define USE_P 698 +#define USER 699 +#define USING 700 +#define VACUUM 701 +#define VALID 702 +#define VALIDATE 703 +#define VALIDATOR 704 +#define VALUE_P 705 +#define VALUES 706 +#define VARCHAR 707 +#define VARIADIC 708 +#define VARYING 709 +#define VERBOSE 710 +#define VERSION_P 711 +#define VIEW 712 +#define VIEWS 713 +#define VIRTUAL 714 +#define VOLATILE 715 +#define WHEN 716 +#define WHERE 717 +#define WHITESPACE_P 718 +#define WINDOW 719 +#define WITH 720 +#define WITHIN 721 +#define WITHOUT 722 +#define WORK 723 +#define WRAPPER 724 +#define WRITE_P 725 +#define XML_P 726 +#define XMLATTRIBUTES 727 +#define XMLCONCAT 728 +#define XMLELEMENT 729 +#define XMLEXISTS 730 +#define XMLFOREST 731 +#define XMLNAMESPACES 732 +#define XMLPARSE 733 +#define XMLPI 734 +#define XMLROOT 735 +#define XMLSERIALIZE 736 +#define XMLTABLE 737 +#define YEAR_P 738 +#define YEARS_P 739 +#define YES_P 740 +#define ZONE 741 +#define NOT_LA 742 +#define NULLS_LA 743 +#define WITH_LA 744 +#define POSTFIXOP 745 +#define UMINUS 746 @@ -1069,7 +1075,7 @@ typedef union YYSTYPE PGInsertColumnOrder bynameorposition; } /* Line 1529 of yacc.c. */ -#line 1073 "third_party/libpg_query/grammar/grammar_out.hpp" +#line 1079 "third_party/libpg_query/grammar/grammar_out.hpp" YYSTYPE; # define yystype YYSTYPE /* obsolescent; will be withdrawn */ # define YYSTYPE_IS_DECLARED 1 diff --git a/src/duckdb/third_party/libpg_query/include/parser/kwlist.hpp b/src/duckdb/third_party/libpg_query/include/parser/kwlist.hpp index 3278f82b1..883498e46 100644 --- a/src/duckdb/third_party/libpg_query/include/parser/kwlist.hpp +++ b/src/duckdb/third_party/libpg_query/include/parser/kwlist.hpp @@ -160,6 +160,7 @@ PG_KEYWORD("granted", GRANTED, UNRESERVED_KEYWORD) PG_KEYWORD("group", GROUP_P, RESERVED_KEYWORD) PG_KEYWORD("grouping", GROUPING, COL_NAME_KEYWORD) PG_KEYWORD("grouping_id", GROUPING_ID, COL_NAME_KEYWORD) +PG_KEYWORD("groups", GROUPS, UNRESERVED_KEYWORD) PG_KEYWORD("handler", HANDLER, UNRESERVED_KEYWORD) PG_KEYWORD("having", HAVING, RESERVED_KEYWORD) PG_KEYWORD("header", HEADER_P, UNRESERVED_KEYWORD) @@ -271,6 +272,7 @@ PG_KEYWORD("options", OPTIONS, UNRESERVED_KEYWORD) PG_KEYWORD("or", OR, RESERVED_KEYWORD) PG_KEYWORD("order", ORDER, RESERVED_KEYWORD) PG_KEYWORD("ordinality", ORDINALITY, UNRESERVED_KEYWORD) +PG_KEYWORD("others", OTHERS, UNRESERVED_KEYWORD) PG_KEYWORD("out", OUT_P, COL_NAME_KEYWORD) PG_KEYWORD("outer", OUTER_P, TYPE_FUNC_NAME_KEYWORD) PG_KEYWORD("over", OVER, UNRESERVED_KEYWORD) @@ -395,6 +397,7 @@ PG_KEYWORD("template", TEMPLATE, UNRESERVED_KEYWORD) PG_KEYWORD("temporary", TEMPORARY, UNRESERVED_KEYWORD) PG_KEYWORD("text", TEXT_P, UNRESERVED_KEYWORD) PG_KEYWORD("then", THEN, RESERVED_KEYWORD) +PG_KEYWORD("ties", TIES, UNRESERVED_KEYWORD) PG_KEYWORD("time", TIME, COL_NAME_KEYWORD) PG_KEYWORD("timestamp", TIMESTAMP, COL_NAME_KEYWORD) PG_KEYWORD("to", TO, RESERVED_KEYWORD) diff --git a/src/duckdb/third_party/libpg_query/src_backend_parser_gram.cpp b/src/duckdb/third_party/libpg_query/src_backend_parser_gram.cpp index 208ea1a11..9fee5c774 100644 --- a/src/duckdb/third_party/libpg_query/src_backend_parser_gram.cpp +++ b/src/duckdb/third_party/libpg_query/src_backend_parser_gram.cpp @@ -250,316 +250,319 @@ GROUP_P = 431, GROUPING = 432, GROUPING_ID = 433, - HANDLER = 434, - HAVING = 435, - HEADER_P = 436, - HOLD = 437, - HOUR_P = 438, - HOURS_P = 439, - IDENTITY_P = 440, - IF_P = 441, - IGNORE_P = 442, - ILIKE = 443, - IMMEDIATE = 444, - IMMUTABLE = 445, - IMPLICIT_P = 446, - IMPORT_P = 447, - IN_P = 448, - INCLUDE_P = 449, - INCLUDING = 450, - INCREMENT = 451, - INDEX = 452, - INDEXES = 453, - INHERIT = 454, - INHERITS = 455, - INITIALLY = 456, - INLINE_P = 457, - INNER_P = 458, - INOUT = 459, - INPUT_P = 460, - INSENSITIVE = 461, - INSERT = 462, - INSTALL = 463, - INSTEAD = 464, - INT_P = 465, - INTEGER = 466, - INTERSECT = 467, - INTERVAL = 468, - INTO = 469, - INVOKER = 470, - IS = 471, - ISNULL = 472, - ISOLATION = 473, - JOIN = 474, - JSON = 475, - KEY = 476, - LABEL = 477, - LANGUAGE = 478, - LARGE_P = 479, - LAST_P = 480, - LATERAL_P = 481, - LEADING = 482, - LEAKPROOF = 483, - LEFT = 484, - LEVEL = 485, - LIKE = 486, - LIMIT = 487, - LISTEN = 488, - LOAD = 489, - LOCAL = 490, - LOCATION = 491, - LOCK_P = 492, - LOCKED = 493, - LOGGED = 494, - MACRO = 495, - MAP = 496, - MAPPING = 497, - MATCH = 498, - MATERIALIZED = 499, - MAXVALUE = 500, - METHOD = 501, - MICROSECOND_P = 502, - MICROSECONDS_P = 503, - MILLISECOND_P = 504, - MILLISECONDS_P = 505, - MINUTE_P = 506, - MINUTES_P = 507, - MINVALUE = 508, - MODE = 509, - MONTH_P = 510, - MONTHS_P = 511, - MOVE = 512, - NAME_P = 513, - NAMES = 514, - NATIONAL = 515, - NATURAL = 516, - NCHAR = 517, - NEW = 518, - NEXT = 519, - NO = 520, - NONE = 521, - NOT = 522, - NOTHING = 523, - NOTIFY = 524, - NOTNULL = 525, - NOWAIT = 526, - NULL_P = 527, - NULLIF = 528, - NULLS_P = 529, - NUMERIC = 530, - OBJECT_P = 531, - OF = 532, - OFF = 533, - OFFSET = 534, - OIDS = 535, - OLD = 536, - ON = 537, - ONLY = 538, - OPERATOR = 539, - OPTION = 540, - OPTIONS = 541, - OR = 542, - ORDER = 543, - ORDINALITY = 544, - OUT_P = 545, - OUTER_P = 546, - OVER = 547, - OVERLAPS = 548, - OVERLAY = 549, - OVERRIDING = 550, - OWNED = 551, - OWNER = 552, - PARALLEL = 553, - PARSER = 554, - PARTIAL = 555, - PARTITION = 556, - PASSING = 557, - PASSWORD = 558, - PERCENT = 559, - PIVOT = 560, - PIVOT_LONGER = 561, - PIVOT_WIDER = 562, - PLACING = 563, - PLANS = 564, - POLICY = 565, - POSITION = 566, - POSITIONAL = 567, - PRAGMA_P = 568, - PRECEDING = 569, - PRECISION = 570, - PREPARE = 571, - PREPARED = 572, - PRESERVE = 573, - PRIMARY = 574, - PRIOR = 575, - PRIVILEGES = 576, - PROCEDURAL = 577, - PROCEDURE = 578, - PROGRAM = 579, - PUBLICATION = 580, - QUALIFY = 581, - QUOTE = 582, - RANGE = 583, - READ_P = 584, - REAL = 585, - REASSIGN = 586, - RECHECK = 587, - RECURSIVE = 588, - REF = 589, - REFERENCES = 590, - REFERENCING = 591, - REFRESH = 592, - REINDEX = 593, - RELATIVE_P = 594, - RELEASE = 595, - RENAME = 596, - REPEATABLE = 597, - REPLACE = 598, - REPLICA = 599, - RESET = 600, - RESPECT_P = 601, - RESTART = 602, - RESTRICT = 603, - RETURNING = 604, - RETURNS = 605, - REVOKE = 606, - RIGHT = 607, - ROLE = 608, - ROLLBACK = 609, - ROLLUP = 610, - ROW = 611, - ROWS = 612, - RULE = 613, - SAMPLE = 614, - SAVEPOINT = 615, - SCHEMA = 616, - SCHEMAS = 617, - SCROLL = 618, - SEARCH = 619, - SECOND_P = 620, - SECONDS_P = 621, - SECURITY = 622, - SELECT = 623, - SEMI = 624, - SEQUENCE = 625, - SEQUENCES = 626, - SERIALIZABLE = 627, - SERVER = 628, - SESSION = 629, - SET = 630, - SETOF = 631, - SETS = 632, - SHARE = 633, - SHOW = 634, - SIMILAR = 635, - SIMPLE = 636, - SKIP = 637, - SMALLINT = 638, - SNAPSHOT = 639, - SOME = 640, - SQL_P = 641, - STABLE = 642, - STANDALONE_P = 643, - START = 644, - STATEMENT = 645, - STATISTICS = 646, - STDIN = 647, - STDOUT = 648, - STORAGE = 649, - STORED = 650, - STRICT_P = 651, - STRIP_P = 652, - STRUCT = 653, - SUBSCRIPTION = 654, - SUBSTRING = 655, - SUMMARIZE = 656, - SYMMETRIC = 657, - SYSID = 658, - SYSTEM_P = 659, - TABLE = 660, - TABLES = 661, - TABLESAMPLE = 662, - TABLESPACE = 663, - TEMP = 664, - TEMPLATE = 665, - TEMPORARY = 666, - TEXT_P = 667, - THEN = 668, - TIME = 669, - TIMESTAMP = 670, - TO = 671, - TRAILING = 672, - TRANSACTION = 673, - TRANSFORM = 674, - TREAT = 675, - TRIGGER = 676, - TRIM = 677, - TRUE_P = 678, - TRUNCATE = 679, - TRUSTED = 680, - TRY_CAST = 681, - TYPE_P = 682, - TYPES_P = 683, - UNBOUNDED = 684, - UNCOMMITTED = 685, - UNENCRYPTED = 686, - UNION = 687, - UNIQUE = 688, - UNKNOWN = 689, - UNLISTEN = 690, - UNLOGGED = 691, - UNPIVOT = 692, - UNTIL = 693, - UPDATE = 694, - USE_P = 695, - USER = 696, - USING = 697, - VACUUM = 698, - VALID = 699, - VALIDATE = 700, - VALIDATOR = 701, - VALUE_P = 702, - VALUES = 703, - VARCHAR = 704, - VARIADIC = 705, - VARYING = 706, - VERBOSE = 707, - VERSION_P = 708, - VIEW = 709, - VIEWS = 710, - VIRTUAL = 711, - VOLATILE = 712, - WHEN = 713, - WHERE = 714, - WHITESPACE_P = 715, - WINDOW = 716, - WITH = 717, - WITHIN = 718, - WITHOUT = 719, - WORK = 720, - WRAPPER = 721, - WRITE_P = 722, - XML_P = 723, - XMLATTRIBUTES = 724, - XMLCONCAT = 725, - XMLELEMENT = 726, - XMLEXISTS = 727, - XMLFOREST = 728, - XMLNAMESPACES = 729, - XMLPARSE = 730, - XMLPI = 731, - XMLROOT = 732, - XMLSERIALIZE = 733, - XMLTABLE = 734, - YEAR_P = 735, - YEARS_P = 736, - YES_P = 737, - ZONE = 738, - NOT_LA = 739, - NULLS_LA = 740, - WITH_LA = 741, - POSTFIXOP = 742, - UMINUS = 743 + GROUPS = 434, + HANDLER = 435, + HAVING = 436, + HEADER_P = 437, + HOLD = 438, + HOUR_P = 439, + HOURS_P = 440, + IDENTITY_P = 441, + IF_P = 442, + IGNORE_P = 443, + ILIKE = 444, + IMMEDIATE = 445, + IMMUTABLE = 446, + IMPLICIT_P = 447, + IMPORT_P = 448, + IN_P = 449, + INCLUDE_P = 450, + INCLUDING = 451, + INCREMENT = 452, + INDEX = 453, + INDEXES = 454, + INHERIT = 455, + INHERITS = 456, + INITIALLY = 457, + INLINE_P = 458, + INNER_P = 459, + INOUT = 460, + INPUT_P = 461, + INSENSITIVE = 462, + INSERT = 463, + INSTALL = 464, + INSTEAD = 465, + INT_P = 466, + INTEGER = 467, + INTERSECT = 468, + INTERVAL = 469, + INTO = 470, + INVOKER = 471, + IS = 472, + ISNULL = 473, + ISOLATION = 474, + JOIN = 475, + JSON = 476, + KEY = 477, + LABEL = 478, + LANGUAGE = 479, + LARGE_P = 480, + LAST_P = 481, + LATERAL_P = 482, + LEADING = 483, + LEAKPROOF = 484, + LEFT = 485, + LEVEL = 486, + LIKE = 487, + LIMIT = 488, + LISTEN = 489, + LOAD = 490, + LOCAL = 491, + LOCATION = 492, + LOCK_P = 493, + LOCKED = 494, + LOGGED = 495, + MACRO = 496, + MAP = 497, + MAPPING = 498, + MATCH = 499, + MATERIALIZED = 500, + MAXVALUE = 501, + METHOD = 502, + MICROSECOND_P = 503, + MICROSECONDS_P = 504, + MILLISECOND_P = 505, + MILLISECONDS_P = 506, + MINUTE_P = 507, + MINUTES_P = 508, + MINVALUE = 509, + MODE = 510, + MONTH_P = 511, + MONTHS_P = 512, + MOVE = 513, + NAME_P = 514, + NAMES = 515, + NATIONAL = 516, + NATURAL = 517, + NCHAR = 518, + NEW = 519, + NEXT = 520, + NO = 521, + NONE = 522, + NOT = 523, + NOTHING = 524, + NOTIFY = 525, + NOTNULL = 526, + NOWAIT = 527, + NULL_P = 528, + NULLIF = 529, + NULLS_P = 530, + NUMERIC = 531, + OBJECT_P = 532, + OF = 533, + OFF = 534, + OFFSET = 535, + OIDS = 536, + OLD = 537, + ON = 538, + ONLY = 539, + OPERATOR = 540, + OPTION = 541, + OPTIONS = 542, + OR = 543, + ORDER = 544, + ORDINALITY = 545, + OTHERS = 546, + OUT_P = 547, + OUTER_P = 548, + OVER = 549, + OVERLAPS = 550, + OVERLAY = 551, + OVERRIDING = 552, + OWNED = 553, + OWNER = 554, + PARALLEL = 555, + PARSER = 556, + PARTIAL = 557, + PARTITION = 558, + PASSING = 559, + PASSWORD = 560, + PERCENT = 561, + PIVOT = 562, + PIVOT_LONGER = 563, + PIVOT_WIDER = 564, + PLACING = 565, + PLANS = 566, + POLICY = 567, + POSITION = 568, + POSITIONAL = 569, + PRAGMA_P = 570, + PRECEDING = 571, + PRECISION = 572, + PREPARE = 573, + PREPARED = 574, + PRESERVE = 575, + PRIMARY = 576, + PRIOR = 577, + PRIVILEGES = 578, + PROCEDURAL = 579, + PROCEDURE = 580, + PROGRAM = 581, + PUBLICATION = 582, + QUALIFY = 583, + QUOTE = 584, + RANGE = 585, + READ_P = 586, + REAL = 587, + REASSIGN = 588, + RECHECK = 589, + RECURSIVE = 590, + REF = 591, + REFERENCES = 592, + REFERENCING = 593, + REFRESH = 594, + REINDEX = 595, + RELATIVE_P = 596, + RELEASE = 597, + RENAME = 598, + REPEATABLE = 599, + REPLACE = 600, + REPLICA = 601, + RESET = 602, + RESPECT_P = 603, + RESTART = 604, + RESTRICT = 605, + RETURNING = 606, + RETURNS = 607, + REVOKE = 608, + RIGHT = 609, + ROLE = 610, + ROLLBACK = 611, + ROLLUP = 612, + ROW = 613, + ROWS = 614, + RULE = 615, + SAMPLE = 616, + SAVEPOINT = 617, + SCHEMA = 618, + SCHEMAS = 619, + SCROLL = 620, + SEARCH = 621, + SECOND_P = 622, + SECONDS_P = 623, + SECURITY = 624, + SELECT = 625, + SEMI = 626, + SEQUENCE = 627, + SEQUENCES = 628, + SERIALIZABLE = 629, + SERVER = 630, + SESSION = 631, + SET = 632, + SETOF = 633, + SETS = 634, + SHARE = 635, + SHOW = 636, + SIMILAR = 637, + SIMPLE = 638, + SKIP = 639, + SMALLINT = 640, + SNAPSHOT = 641, + SOME = 642, + SQL_P = 643, + STABLE = 644, + STANDALONE_P = 645, + START = 646, + STATEMENT = 647, + STATISTICS = 648, + STDIN = 649, + STDOUT = 650, + STORAGE = 651, + STORED = 652, + STRICT_P = 653, + STRIP_P = 654, + STRUCT = 655, + SUBSCRIPTION = 656, + SUBSTRING = 657, + SUMMARIZE = 658, + SYMMETRIC = 659, + SYSID = 660, + SYSTEM_P = 661, + TABLE = 662, + TABLES = 663, + TABLESAMPLE = 664, + TABLESPACE = 665, + TEMP = 666, + TEMPLATE = 667, + TEMPORARY = 668, + TEXT_P = 669, + THEN = 670, + TIES = 671, + TIME = 672, + TIMESTAMP = 673, + TO = 674, + TRAILING = 675, + TRANSACTION = 676, + TRANSFORM = 677, + TREAT = 678, + TRIGGER = 679, + TRIM = 680, + TRUE_P = 681, + TRUNCATE = 682, + TRUSTED = 683, + TRY_CAST = 684, + TYPE_P = 685, + TYPES_P = 686, + UNBOUNDED = 687, + UNCOMMITTED = 688, + UNENCRYPTED = 689, + UNION = 690, + UNIQUE = 691, + UNKNOWN = 692, + UNLISTEN = 693, + UNLOGGED = 694, + UNPIVOT = 695, + UNTIL = 696, + UPDATE = 697, + USE_P = 698, + USER = 699, + USING = 700, + VACUUM = 701, + VALID = 702, + VALIDATE = 703, + VALIDATOR = 704, + VALUE_P = 705, + VALUES = 706, + VARCHAR = 707, + VARIADIC = 708, + VARYING = 709, + VERBOSE = 710, + VERSION_P = 711, + VIEW = 712, + VIEWS = 713, + VIRTUAL = 714, + VOLATILE = 715, + WHEN = 716, + WHERE = 717, + WHITESPACE_P = 718, + WINDOW = 719, + WITH = 720, + WITHIN = 721, + WITHOUT = 722, + WORK = 723, + WRAPPER = 724, + WRITE_P = 725, + XML_P = 726, + XMLATTRIBUTES = 727, + XMLCONCAT = 728, + XMLELEMENT = 729, + XMLEXISTS = 730, + XMLFOREST = 731, + XMLNAMESPACES = 732, + XMLPARSE = 733, + XMLPI = 734, + XMLROOT = 735, + XMLSERIALIZE = 736, + XMLTABLE = 737, + YEAR_P = 738, + YEARS_P = 739, + YES_P = 740, + ZONE = 741, + NOT_LA = 742, + NULLS_LA = 743, + WITH_LA = 744, + POSTFIXOP = 745, + UMINUS = 746 }; #endif /* Tokens. */ @@ -739,316 +742,319 @@ #define GROUP_P 431 #define GROUPING 432 #define GROUPING_ID 433 -#define HANDLER 434 -#define HAVING 435 -#define HEADER_P 436 -#define HOLD 437 -#define HOUR_P 438 -#define HOURS_P 439 -#define IDENTITY_P 440 -#define IF_P 441 -#define IGNORE_P 442 -#define ILIKE 443 -#define IMMEDIATE 444 -#define IMMUTABLE 445 -#define IMPLICIT_P 446 -#define IMPORT_P 447 -#define IN_P 448 -#define INCLUDE_P 449 -#define INCLUDING 450 -#define INCREMENT 451 -#define INDEX 452 -#define INDEXES 453 -#define INHERIT 454 -#define INHERITS 455 -#define INITIALLY 456 -#define INLINE_P 457 -#define INNER_P 458 -#define INOUT 459 -#define INPUT_P 460 -#define INSENSITIVE 461 -#define INSERT 462 -#define INSTALL 463 -#define INSTEAD 464 -#define INT_P 465 -#define INTEGER 466 -#define INTERSECT 467 -#define INTERVAL 468 -#define INTO 469 -#define INVOKER 470 -#define IS 471 -#define ISNULL 472 -#define ISOLATION 473 -#define JOIN 474 -#define JSON 475 -#define KEY 476 -#define LABEL 477 -#define LANGUAGE 478 -#define LARGE_P 479 -#define LAST_P 480 -#define LATERAL_P 481 -#define LEADING 482 -#define LEAKPROOF 483 -#define LEFT 484 -#define LEVEL 485 -#define LIKE 486 -#define LIMIT 487 -#define LISTEN 488 -#define LOAD 489 -#define LOCAL 490 -#define LOCATION 491 -#define LOCK_P 492 -#define LOCKED 493 -#define LOGGED 494 -#define MACRO 495 -#define MAP 496 -#define MAPPING 497 -#define MATCH 498 -#define MATERIALIZED 499 -#define MAXVALUE 500 -#define METHOD 501 -#define MICROSECOND_P 502 -#define MICROSECONDS_P 503 -#define MILLISECOND_P 504 -#define MILLISECONDS_P 505 -#define MINUTE_P 506 -#define MINUTES_P 507 -#define MINVALUE 508 -#define MODE 509 -#define MONTH_P 510 -#define MONTHS_P 511 -#define MOVE 512 -#define NAME_P 513 -#define NAMES 514 -#define NATIONAL 515 -#define NATURAL 516 -#define NCHAR 517 -#define NEW 518 -#define NEXT 519 -#define NO 520 -#define NONE 521 -#define NOT 522 -#define NOTHING 523 -#define NOTIFY 524 -#define NOTNULL 525 -#define NOWAIT 526 -#define NULL_P 527 -#define NULLIF 528 -#define NULLS_P 529 -#define NUMERIC 530 -#define OBJECT_P 531 -#define OF 532 -#define OFF 533 -#define OFFSET 534 -#define OIDS 535 -#define OLD 536 -#define ON 537 -#define ONLY 538 -#define OPERATOR 539 -#define OPTION 540 -#define OPTIONS 541 -#define OR 542 -#define ORDER 543 -#define ORDINALITY 544 -#define OUT_P 545 -#define OUTER_P 546 -#define OVER 547 -#define OVERLAPS 548 -#define OVERLAY 549 -#define OVERRIDING 550 -#define OWNED 551 -#define OWNER 552 -#define PARALLEL 553 -#define PARSER 554 -#define PARTIAL 555 -#define PARTITION 556 -#define PASSING 557 -#define PASSWORD 558 -#define PERCENT 559 -#define PIVOT 560 -#define PIVOT_LONGER 561 -#define PIVOT_WIDER 562 -#define PLACING 563 -#define PLANS 564 -#define POLICY 565 -#define POSITION 566 -#define POSITIONAL 567 -#define PRAGMA_P 568 -#define PRECEDING 569 -#define PRECISION 570 -#define PREPARE 571 -#define PREPARED 572 -#define PRESERVE 573 -#define PRIMARY 574 -#define PRIOR 575 -#define PRIVILEGES 576 -#define PROCEDURAL 577 -#define PROCEDURE 578 -#define PROGRAM 579 -#define PUBLICATION 580 -#define QUALIFY 581 -#define QUOTE 582 -#define RANGE 583 -#define READ_P 584 -#define REAL 585 -#define REASSIGN 586 -#define RECHECK 587 -#define RECURSIVE 588 -#define REF 589 -#define REFERENCES 590 -#define REFERENCING 591 -#define REFRESH 592 -#define REINDEX 593 -#define RELATIVE_P 594 -#define RELEASE 595 -#define RENAME 596 -#define REPEATABLE 597 -#define REPLACE 598 -#define REPLICA 599 -#define RESET 600 -#define RESPECT_P 601 -#define RESTART 602 -#define RESTRICT 603 -#define RETURNING 604 -#define RETURNS 605 -#define REVOKE 606 -#define RIGHT 607 -#define ROLE 608 -#define ROLLBACK 609 -#define ROLLUP 610 -#define ROW 611 -#define ROWS 612 -#define RULE 613 -#define SAMPLE 614 -#define SAVEPOINT 615 -#define SCHEMA 616 -#define SCHEMAS 617 -#define SCROLL 618 -#define SEARCH 619 -#define SECOND_P 620 -#define SECONDS_P 621 -#define SECURITY 622 -#define SELECT 623 -#define SEMI 624 -#define SEQUENCE 625 -#define SEQUENCES 626 -#define SERIALIZABLE 627 -#define SERVER 628 -#define SESSION 629 -#define SET 630 -#define SETOF 631 -#define SETS 632 -#define SHARE 633 -#define SHOW 634 -#define SIMILAR 635 -#define SIMPLE 636 -#define SKIP 637 -#define SMALLINT 638 -#define SNAPSHOT 639 -#define SOME 640 -#define SQL_P 641 -#define STABLE 642 -#define STANDALONE_P 643 -#define START 644 -#define STATEMENT 645 -#define STATISTICS 646 -#define STDIN 647 -#define STDOUT 648 -#define STORAGE 649 -#define STORED 650 -#define STRICT_P 651 -#define STRIP_P 652 -#define STRUCT 653 -#define SUBSCRIPTION 654 -#define SUBSTRING 655 -#define SUMMARIZE 656 -#define SYMMETRIC 657 -#define SYSID 658 -#define SYSTEM_P 659 -#define TABLE 660 -#define TABLES 661 -#define TABLESAMPLE 662 -#define TABLESPACE 663 -#define TEMP 664 -#define TEMPLATE 665 -#define TEMPORARY 666 -#define TEXT_P 667 -#define THEN 668 -#define TIME 669 -#define TIMESTAMP 670 -#define TO 671 -#define TRAILING 672 -#define TRANSACTION 673 -#define TRANSFORM 674 -#define TREAT 675 -#define TRIGGER 676 -#define TRIM 677 -#define TRUE_P 678 -#define TRUNCATE 679 -#define TRUSTED 680 -#define TRY_CAST 681 -#define TYPE_P 682 -#define TYPES_P 683 -#define UNBOUNDED 684 -#define UNCOMMITTED 685 -#define UNENCRYPTED 686 -#define UNION 687 -#define UNIQUE 688 -#define UNKNOWN 689 -#define UNLISTEN 690 -#define UNLOGGED 691 -#define UNPIVOT 692 -#define UNTIL 693 -#define UPDATE 694 -#define USE_P 695 -#define USER 696 -#define USING 697 -#define VACUUM 698 -#define VALID 699 -#define VALIDATE 700 -#define VALIDATOR 701 -#define VALUE_P 702 -#define VALUES 703 -#define VARCHAR 704 -#define VARIADIC 705 -#define VARYING 706 -#define VERBOSE 707 -#define VERSION_P 708 -#define VIEW 709 -#define VIEWS 710 -#define VIRTUAL 711 -#define VOLATILE 712 -#define WHEN 713 -#define WHERE 714 -#define WHITESPACE_P 715 -#define WINDOW 716 -#define WITH 717 -#define WITHIN 718 -#define WITHOUT 719 -#define WORK 720 -#define WRAPPER 721 -#define WRITE_P 722 -#define XML_P 723 -#define XMLATTRIBUTES 724 -#define XMLCONCAT 725 -#define XMLELEMENT 726 -#define XMLEXISTS 727 -#define XMLFOREST 728 -#define XMLNAMESPACES 729 -#define XMLPARSE 730 -#define XMLPI 731 -#define XMLROOT 732 -#define XMLSERIALIZE 733 -#define XMLTABLE 734 -#define YEAR_P 735 -#define YEARS_P 736 -#define YES_P 737 -#define ZONE 738 -#define NOT_LA 739 -#define NULLS_LA 740 -#define WITH_LA 741 -#define POSTFIXOP 742 -#define UMINUS 743 +#define GROUPS 434 +#define HANDLER 435 +#define HAVING 436 +#define HEADER_P 437 +#define HOLD 438 +#define HOUR_P 439 +#define HOURS_P 440 +#define IDENTITY_P 441 +#define IF_P 442 +#define IGNORE_P 443 +#define ILIKE 444 +#define IMMEDIATE 445 +#define IMMUTABLE 446 +#define IMPLICIT_P 447 +#define IMPORT_P 448 +#define IN_P 449 +#define INCLUDE_P 450 +#define INCLUDING 451 +#define INCREMENT 452 +#define INDEX 453 +#define INDEXES 454 +#define INHERIT 455 +#define INHERITS 456 +#define INITIALLY 457 +#define INLINE_P 458 +#define INNER_P 459 +#define INOUT 460 +#define INPUT_P 461 +#define INSENSITIVE 462 +#define INSERT 463 +#define INSTALL 464 +#define INSTEAD 465 +#define INT_P 466 +#define INTEGER 467 +#define INTERSECT 468 +#define INTERVAL 469 +#define INTO 470 +#define INVOKER 471 +#define IS 472 +#define ISNULL 473 +#define ISOLATION 474 +#define JOIN 475 +#define JSON 476 +#define KEY 477 +#define LABEL 478 +#define LANGUAGE 479 +#define LARGE_P 480 +#define LAST_P 481 +#define LATERAL_P 482 +#define LEADING 483 +#define LEAKPROOF 484 +#define LEFT 485 +#define LEVEL 486 +#define LIKE 487 +#define LIMIT 488 +#define LISTEN 489 +#define LOAD 490 +#define LOCAL 491 +#define LOCATION 492 +#define LOCK_P 493 +#define LOCKED 494 +#define LOGGED 495 +#define MACRO 496 +#define MAP 497 +#define MAPPING 498 +#define MATCH 499 +#define MATERIALIZED 500 +#define MAXVALUE 501 +#define METHOD 502 +#define MICROSECOND_P 503 +#define MICROSECONDS_P 504 +#define MILLISECOND_P 505 +#define MILLISECONDS_P 506 +#define MINUTE_P 507 +#define MINUTES_P 508 +#define MINVALUE 509 +#define MODE 510 +#define MONTH_P 511 +#define MONTHS_P 512 +#define MOVE 513 +#define NAME_P 514 +#define NAMES 515 +#define NATIONAL 516 +#define NATURAL 517 +#define NCHAR 518 +#define NEW 519 +#define NEXT 520 +#define NO 521 +#define NONE 522 +#define NOT 523 +#define NOTHING 524 +#define NOTIFY 525 +#define NOTNULL 526 +#define NOWAIT 527 +#define NULL_P 528 +#define NULLIF 529 +#define NULLS_P 530 +#define NUMERIC 531 +#define OBJECT_P 532 +#define OF 533 +#define OFF 534 +#define OFFSET 535 +#define OIDS 536 +#define OLD 537 +#define ON 538 +#define ONLY 539 +#define OPERATOR 540 +#define OPTION 541 +#define OPTIONS 542 +#define OR 543 +#define ORDER 544 +#define ORDINALITY 545 +#define OTHERS 546 +#define OUT_P 547 +#define OUTER_P 548 +#define OVER 549 +#define OVERLAPS 550 +#define OVERLAY 551 +#define OVERRIDING 552 +#define OWNED 553 +#define OWNER 554 +#define PARALLEL 555 +#define PARSER 556 +#define PARTIAL 557 +#define PARTITION 558 +#define PASSING 559 +#define PASSWORD 560 +#define PERCENT 561 +#define PIVOT 562 +#define PIVOT_LONGER 563 +#define PIVOT_WIDER 564 +#define PLACING 565 +#define PLANS 566 +#define POLICY 567 +#define POSITION 568 +#define POSITIONAL 569 +#define PRAGMA_P 570 +#define PRECEDING 571 +#define PRECISION 572 +#define PREPARE 573 +#define PREPARED 574 +#define PRESERVE 575 +#define PRIMARY 576 +#define PRIOR 577 +#define PRIVILEGES 578 +#define PROCEDURAL 579 +#define PROCEDURE 580 +#define PROGRAM 581 +#define PUBLICATION 582 +#define QUALIFY 583 +#define QUOTE 584 +#define RANGE 585 +#define READ_P 586 +#define REAL 587 +#define REASSIGN 588 +#define RECHECK 589 +#define RECURSIVE 590 +#define REF 591 +#define REFERENCES 592 +#define REFERENCING 593 +#define REFRESH 594 +#define REINDEX 595 +#define RELATIVE_P 596 +#define RELEASE 597 +#define RENAME 598 +#define REPEATABLE 599 +#define REPLACE 600 +#define REPLICA 601 +#define RESET 602 +#define RESPECT_P 603 +#define RESTART 604 +#define RESTRICT 605 +#define RETURNING 606 +#define RETURNS 607 +#define REVOKE 608 +#define RIGHT 609 +#define ROLE 610 +#define ROLLBACK 611 +#define ROLLUP 612 +#define ROW 613 +#define ROWS 614 +#define RULE 615 +#define SAMPLE 616 +#define SAVEPOINT 617 +#define SCHEMA 618 +#define SCHEMAS 619 +#define SCROLL 620 +#define SEARCH 621 +#define SECOND_P 622 +#define SECONDS_P 623 +#define SECURITY 624 +#define SELECT 625 +#define SEMI 626 +#define SEQUENCE 627 +#define SEQUENCES 628 +#define SERIALIZABLE 629 +#define SERVER 630 +#define SESSION 631 +#define SET 632 +#define SETOF 633 +#define SETS 634 +#define SHARE 635 +#define SHOW 636 +#define SIMILAR 637 +#define SIMPLE 638 +#define SKIP 639 +#define SMALLINT 640 +#define SNAPSHOT 641 +#define SOME 642 +#define SQL_P 643 +#define STABLE 644 +#define STANDALONE_P 645 +#define START 646 +#define STATEMENT 647 +#define STATISTICS 648 +#define STDIN 649 +#define STDOUT 650 +#define STORAGE 651 +#define STORED 652 +#define STRICT_P 653 +#define STRIP_P 654 +#define STRUCT 655 +#define SUBSCRIPTION 656 +#define SUBSTRING 657 +#define SUMMARIZE 658 +#define SYMMETRIC 659 +#define SYSID 660 +#define SYSTEM_P 661 +#define TABLE 662 +#define TABLES 663 +#define TABLESAMPLE 664 +#define TABLESPACE 665 +#define TEMP 666 +#define TEMPLATE 667 +#define TEMPORARY 668 +#define TEXT_P 669 +#define THEN 670 +#define TIES 671 +#define TIME 672 +#define TIMESTAMP 673 +#define TO 674 +#define TRAILING 675 +#define TRANSACTION 676 +#define TRANSFORM 677 +#define TREAT 678 +#define TRIGGER 679 +#define TRIM 680 +#define TRUE_P 681 +#define TRUNCATE 682 +#define TRUSTED 683 +#define TRY_CAST 684 +#define TYPE_P 685 +#define TYPES_P 686 +#define UNBOUNDED 687 +#define UNCOMMITTED 688 +#define UNENCRYPTED 689 +#define UNION 690 +#define UNIQUE 691 +#define UNKNOWN 692 +#define UNLISTEN 693 +#define UNLOGGED 694 +#define UNPIVOT 695 +#define UNTIL 696 +#define UPDATE 697 +#define USE_P 698 +#define USER 699 +#define USING 700 +#define VACUUM 701 +#define VALID 702 +#define VALIDATE 703 +#define VALIDATOR 704 +#define VALUE_P 705 +#define VALUES 706 +#define VARCHAR 707 +#define VARIADIC 708 +#define VARYING 709 +#define VERBOSE 710 +#define VERSION_P 711 +#define VIEW 712 +#define VIEWS 713 +#define VIRTUAL 714 +#define VOLATILE 715 +#define WHEN 716 +#define WHERE 717 +#define WHITESPACE_P 718 +#define WINDOW 719 +#define WITH 720 +#define WITHIN 721 +#define WITHOUT 722 +#define WORK 723 +#define WRAPPER 724 +#define WRITE_P 725 +#define XML_P 726 +#define XMLATTRIBUTES 727 +#define XMLCONCAT 728 +#define XMLELEMENT 729 +#define XMLEXISTS 730 +#define XMLFOREST 731 +#define XMLNAMESPACES 732 +#define XMLPARSE 733 +#define XMLPI 734 +#define XMLROOT 735 +#define XMLSERIALIZE 736 +#define XMLTABLE 737 +#define YEAR_P 738 +#define YEARS_P 739 +#define YES_P 740 +#define ZONE 741 +#define NOT_LA 742 +#define NULLS_LA 743 +#define WITH_LA 744 +#define POSTFIXOP 745 +#define UMINUS 746 @@ -1289,7 +1295,7 @@ typedef union YYSTYPE PGInsertColumnOrder bynameorposition; } /* Line 193 of yacc.c. */ -#line 1293 "third_party/libpg_query/grammar/grammar_out.cpp" +#line 1299 "third_party/libpg_query/grammar/grammar_out.cpp" YYSTYPE; # define yystype YYSTYPE /* obsolescent; will be withdrawn */ # define YYSTYPE_IS_DECLARED 1 @@ -1314,7 +1320,7 @@ typedef struct YYLTYPE /* Line 216 of yacc.c. */ -#line 1318 "third_party/libpg_query/grammar/grammar_out.cpp" +#line 1324 "third_party/libpg_query/grammar/grammar_out.cpp" #ifdef short # undef short @@ -1529,22 +1535,22 @@ union yyalloc #endif /* YYFINAL -- State number of the termination state. */ -#define YYFINAL 680 +#define YYFINAL 683 /* YYLAST -- Last index in YYTABLE. */ -#define YYLAST 68852 +#define YYLAST 69371 /* YYNTOKENS -- Number of terminals. */ -#define YYNTOKENS 511 +#define YYNTOKENS 514 /* YYNNTS -- Number of nonterminals. */ -#define YYNNTS 442 +#define YYNNTS 443 /* YYNRULES -- Number of rules. */ -#define YYNRULES 2039 +#define YYNRULES 2048 /* YYNRULES -- Number of states. */ -#define YYNSTATES 3399 +#define YYNSTATES 3414 /* YYTRANSLATE(YYLEX) -- Bison symbol number corresponding to YYLEX. */ #define YYUNDEFTOK 2 -#define YYMAXUTOK 743 +#define YYMAXUTOK 746 #define YYTRANSLATE(YYX) \ ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK) @@ -1555,16 +1561,16 @@ static const yytype_uint16 yytranslate[] = 0, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 2, 505, 506, 495, 2, 2, - 500, 501, 493, 491, 504, 492, 502, 494, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, 510, 503, - 487, 489, 488, 507, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 508, 509, 498, 2, 2, + 503, 504, 496, 494, 507, 495, 505, 497, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, 513, 506, + 490, 492, 491, 510, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 2, 498, 2, 499, 496, 2, 2, 2, 2, 2, + 2, 501, 2, 502, 499, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 508, 2, 509, 2, 2, 2, 2, + 2, 2, 2, 511, 2, 512, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, @@ -1626,7 +1632,7 @@ static const yytype_uint16 yytranslate[] = 455, 456, 457, 458, 459, 460, 461, 462, 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 490, 497 + 485, 486, 487, 488, 489, 493, 500 }; #if YYDEBUG @@ -1736,738 +1742,742 @@ static const yytype_uint16 yyprhs[] = 3499, 3501, 3503, 3509, 3516, 3523, 3528, 3533, 3538, 3543, 3550, 3556, 3562, 3568, 3573, 3580, 3585, 3593, 3603, 3609, 3610, 3616, 3621, 3622, 3624, 3625, 3628, 3629, 3631, 3635, - 3639, 3642, 3645, 3646, 3653, 3655, 3656, 3660, 3661, 3664, - 3667, 3668, 3670, 3675, 3678, 3681, 3684, 3687, 3690, 3695, - 3699, 3701, 3707, 3711, 3713, 3717, 3719, 3722, 3726, 3728, - 3732, 3734, 3737, 3739, 3740, 3742, 3744, 3746, 3748, 3750, - 3752, 3754, 3756, 3758, 3760, 3762, 3764, 3766, 3768, 3770, - 3772, 3774, 3776, 3778, 3780, 3785, 3787, 3792, 3794, 3799, - 3801, 3804, 3806, 3809, 3811, 3814, 3816, 3820, 3822, 3826, - 3828, 3831, 3833, 3837, 3839, 3842, 3844, 3845, 3847, 3851, - 3853, 3857, 3861, 3863, 3867, 3871, 3872, 3874, 3876, 3878, - 3880, 3882, 3884, 3886, 3888, 3890, 3892, 3897, 3901, 3904, - 3908, 3909, 3913, 3917, 3920, 3923, 3925, 3926, 3929, 3932, - 3936, 3939, 3941, 3943, 3947, 3953, 3955, 3958, 3963, 3966, - 3967, 3969, 3970, 3972, 3975, 3979, 3985, 3993, 4001, 4003, - 4004, 4005, 4008, 4009, 4012, 4016, 4020, 4024, 4030, 4038, - 4046, 4047, 4050, 4052, 4053, 4055, 4056, 4058, 4062, 4064, - 4067, 4071, 4074, 4076, 4081, 4084, 4086, 4087, 4091, 4093, - 4097, 4099, 4102, 4107, 4110, 4111, 4113, 4117, 4119, 4123, - 4125, 4128, 4130, 4134, 4136, 4138, 4141, 4143, 4145, 4148, - 4150, 4152, 4155, 4163, 4166, 4172, 4176, 4180, 4182, 4184, - 4186, 4188, 4190, 4192, 4194, 4196, 4198, 4200, 4202, 4204, - 4206, 4208, 4211, 4214, 4218, 4222, 4223, 4225, 4227, 4229, - 4235, 4239, 4240, 4242, 4244, 4246, 4248, 4250, 4255, 4263, - 4270, 4273, 4274, 4276, 4278, 4280, 4282, 4296, 4313, 4315, - 4318, 4319, 4321, 4322, 4324, 4325, 4328, 4329, 4331, 4332, - 4339, 4348, 4355, 4364, 4371, 4380, 4384, 4387, 4389, 4390, - 4395, 4402, 4406, 4409, 4414, 4418, 4424, 4426, 4427, 4429, - 4431, 4432, 4434, 4436, 4438, 4440, 4442, 4444, 4446, 4448, - 4450, 4452, 4454, 4456, 4458, 4460, 4462, 4464, 4466, 4468, - 4470, 4472, 4474, 4476, 4478, 4480, 4482, 4484, 4486, 4488, - 4490, 4492, 4494, 4496, 4498, 4500, 4502, 4504, 4508, 4510, - 4512, 4514, 4516, 4518, 4520, 4523, 4525, 4527, 4530, 4534, - 4538, 4542, 4544, 4548, 4552, 4555, 4559, 4563, 4567, 4571, - 4573, 4575, 4577, 4579, 4583, 4589, 4591, 4593, 4595, 4597, - 4601, 4604, 4607, 4611, 4616, 4622, 4624, 4626, 4628, 4630, - 4635, 4642, 4648, 4653, 4660, 4662, 4664, 4666, 4668, 4670, - 4672, 4673, 4675, 4679, 4681, 4682, 4690, 4694, 4696, 4699, - 4703, 4706, 4707, 4710, 4711, 4714, 4719, 4725, 4729, 4735, - 4737, 4738, 4741, 4742, 4745, 4749, 4753, 4757, 4759, 4761, - 4763, 4766, 4770, 4773, 4776, 4779, 4782, 4786, 4791, 4795, - 4797, 4799, 4801, 4803, 4804, 4806, 4810, 4812, 4816, 4819, - 4829, 4842, 4854, 4867, 4882, 4886, 4891, 4896, 4897, 4905, - 4916, 4926, 4929, 4933, 4934, 4939, 4941, 4943, 4945, 4947, - 4949, 4951, 4953, 4955, 4957, 4959, 4961, 4963, 4965, 4967, - 4969, 4971, 4973, 4975, 4977, 4979, 4981, 4983, 4985, 4987, - 4989, 4991, 4993, 4995, 4997, 4999, 5001, 5003, 5005, 5007, - 5009, 5011, 5013, 5015, 5017, 5019, 5021, 5023, 5025, 5027, - 5029, 5031, 5033, 5035, 5037, 5039, 5041, 5043, 5045, 5047, - 5049, 5051, 5053, 5055, 5057, 5059, 5061, 5063, 5065, 5067, - 5069, 5071, 5073, 5075, 5077, 5079, 5081, 5083, 5085, 5087, - 5089, 5091, 5093, 5095, 5097, 5099, 5101, 5103, 5105, 5107, - 5109, 5111, 5113, 5115, 5117, 5119, 5121, 5123, 5125, 5127, - 5129, 5131, 5133, 5135, 5137, 5139, 5141, 5143, 5145, 5147, - 5149, 5151, 5153, 5155, 5157, 5159, 5161, 5163, 5165, 5167, - 5169, 5171, 5173, 5175, 5177, 5179, 5181, 5183, 5185, 5187, - 5189, 5191, 5193, 5195, 5197, 5199, 5201, 5203, 5205, 5207, - 5209, 5211, 5213, 5215, 5217, 5219, 5221, 5223, 5225, 5227, - 5229, 5231, 5233, 5235, 5237, 5239, 5241, 5243, 5245, 5247, - 5249, 5251, 5253, 5255, 5257, 5259, 5261, 5263, 5265, 5267, - 5269, 5271, 5273, 5275, 5277, 5279, 5281, 5283, 5285, 5287, - 5289, 5291, 5293, 5295, 5297, 5299, 5301, 5303, 5305, 5307, - 5309, 5311, 5313, 5315, 5317, 5319, 5321, 5323, 5325, 5327, - 5329, 5331, 5333, 5335, 5337, 5339, 5341, 5343, 5345, 5347, - 5349, 5351, 5353, 5355, 5357, 5359, 5361, 5363, 5365, 5367, - 5369, 5371, 5373, 5375, 5377, 5379, 5381, 5383, 5385, 5387, - 5389, 5391, 5393, 5395, 5397, 5399, 5401, 5403, 5405, 5407, - 5409, 5411, 5413, 5415, 5417, 5419, 5421, 5423, 5425, 5427, - 5429, 5431, 5433, 5435, 5437, 5439, 5441, 5443, 5445, 5447, - 5449, 5451, 5453, 5455, 5457, 5459, 5461, 5463, 5465, 5467, - 5469, 5471, 5473, 5475, 5477, 5479, 5481, 5483, 5485, 5487, - 5489, 5491, 5493, 5495, 5497, 5499, 5501, 5503, 5505, 5507, - 5509, 5511, 5513, 5515, 5517, 5519, 5521, 5523, 5525, 5527, - 5529, 5531, 5533, 5535, 5537, 5539, 5541, 5543, 5545, 5547, - 5549, 5551, 5553, 5555, 5557, 5559, 5561, 5563, 5565, 5567, - 5569, 5571, 5573, 5575, 5577, 5579, 5581, 5583, 5585, 5587, - 5589, 5591, 5593, 5595, 5597, 5599, 5601, 5603, 5605, 5607, - 5609, 5611, 5613, 5615, 5617, 5619, 5621, 5623, 5625, 5627, - 5629, 5631, 5633, 5635, 5637, 5639, 5641, 5643, 5645, 5647, - 5649, 5651, 5653, 5655, 5657, 5659, 5661, 5663, 5665, 5667, - 5669, 5671, 5673, 5675, 5677, 5679, 5681, 5683, 5685, 5687, - 5689, 5691, 5693, 5695, 5697, 5699, 5701, 5703, 5705, 5707, - 5709, 5711, 5713, 5715, 5717, 5719, 5721, 5723, 5725, 5727, - 5729, 5731, 5733, 5735, 5737, 5739, 5741, 5743, 5745, 5747, - 5749, 5751, 5753, 5755, 5757, 5759, 5761, 5763, 5765, 5767, - 5769, 5771, 5773, 5775, 5777, 5779, 5781, 5783, 5785, 5787, - 5789, 5791, 5793, 5795, 5797, 5799, 5801, 5803, 5805, 5807, - 5809, 5811, 5813, 5815, 5817, 5819, 5821, 5823, 5825, 5827, - 5829, 5831, 5833, 5835, 5837, 5839, 5841, 5843, 5845, 5847, - 5849, 5851, 5853, 5855, 5857, 5859, 5861, 5863, 5865, 5867, - 5869, 5871, 5873, 5875, 5877, 5879, 5881, 5883, 5885, 5887, - 5889, 5891, 5893, 5895, 5897, 5899, 5901, 5903, 5905, 5907, - 5909, 5911, 5913, 5915, 5917, 5919, 5921, 5923, 5925, 5927, - 5929, 5931, 5933, 5935, 5937, 5939, 5941, 5943, 5945, 5947, - 5949, 5951, 5953, 5955, 5957, 5959, 5961, 5963, 5965, 5967, - 5969, 5971, 5973, 5975, 5977, 5979, 5981, 5983, 5985, 5987, - 5989, 5991, 5993, 5995, 5997, 5999, 6001, 6003, 6005, 6007, - 6009, 6011, 6013, 6015, 6017, 6019, 6021, 6023, 6025, 6027, - 6029, 6031, 6033, 6035, 6037, 6039, 6041, 6043, 6045, 6047, - 6049, 6051, 6053, 6055, 6057, 6059, 6061, 6063, 6065, 6067, - 6069, 6071, 6073, 6075, 6077, 6079, 6081, 6083, 6085, 6087, - 6089, 6091, 6093, 6095, 6097, 6099, 6101, 6103, 6105, 6107, - 6109, 6111, 6113, 6115, 6117, 6119, 6121, 6123, 6125, 6127, - 6129, 6131, 6133, 6135, 6137, 6139, 6141, 6143, 6145, 6147 + 3639, 3642, 3645, 3646, 3653, 3655, 3656, 3660, 3661, 3665, + 3669, 3673, 3674, 3676, 3681, 3684, 3687, 3690, 3693, 3696, + 3700, 3703, 3706, 3710, 3711, 3716, 3720, 3722, 3728, 3732, + 3734, 3738, 3740, 3743, 3747, 3749, 3753, 3755, 3758, 3760, + 3761, 3763, 3765, 3767, 3769, 3771, 3773, 3775, 3777, 3779, + 3781, 3783, 3785, 3787, 3789, 3791, 3793, 3795, 3797, 3799, + 3801, 3806, 3808, 3813, 3815, 3820, 3822, 3825, 3827, 3830, + 3832, 3835, 3837, 3841, 3843, 3847, 3849, 3852, 3854, 3858, + 3860, 3863, 3865, 3866, 3868, 3872, 3874, 3878, 3882, 3884, + 3888, 3892, 3893, 3895, 3897, 3899, 3901, 3903, 3905, 3907, + 3909, 3911, 3913, 3918, 3922, 3925, 3929, 3930, 3934, 3938, + 3941, 3944, 3946, 3947, 3950, 3953, 3957, 3960, 3962, 3964, + 3968, 3974, 3976, 3979, 3984, 3987, 3988, 3990, 3991, 3993, + 3996, 4000, 4006, 4014, 4022, 4024, 4025, 4026, 4029, 4030, + 4033, 4037, 4041, 4045, 4051, 4059, 4067, 4068, 4071, 4073, + 4074, 4076, 4077, 4079, 4083, 4085, 4088, 4092, 4095, 4097, + 4102, 4105, 4107, 4108, 4112, 4114, 4118, 4120, 4123, 4128, + 4131, 4132, 4134, 4138, 4140, 4144, 4146, 4149, 4151, 4155, + 4157, 4159, 4162, 4164, 4166, 4169, 4171, 4173, 4176, 4184, + 4187, 4193, 4197, 4201, 4203, 4205, 4207, 4209, 4211, 4213, + 4215, 4217, 4219, 4221, 4223, 4225, 4227, 4229, 4232, 4235, + 4239, 4243, 4244, 4246, 4248, 4250, 4256, 4260, 4261, 4263, + 4265, 4267, 4269, 4271, 4276, 4284, 4291, 4294, 4295, 4297, + 4299, 4301, 4303, 4317, 4334, 4336, 4339, 4340, 4342, 4343, + 4345, 4346, 4349, 4350, 4352, 4353, 4360, 4369, 4376, 4385, + 4392, 4401, 4405, 4408, 4410, 4411, 4416, 4423, 4427, 4430, + 4435, 4439, 4445, 4447, 4448, 4450, 4452, 4453, 4455, 4457, + 4459, 4461, 4463, 4465, 4467, 4469, 4471, 4473, 4475, 4477, + 4479, 4481, 4483, 4485, 4487, 4489, 4491, 4493, 4495, 4497, + 4499, 4501, 4503, 4505, 4507, 4509, 4511, 4513, 4515, 4517, + 4519, 4521, 4523, 4525, 4529, 4531, 4533, 4535, 4537, 4539, + 4541, 4544, 4546, 4548, 4551, 4555, 4559, 4563, 4565, 4569, + 4573, 4576, 4580, 4584, 4588, 4592, 4594, 4596, 4598, 4600, + 4604, 4610, 4612, 4614, 4616, 4618, 4622, 4625, 4628, 4632, + 4637, 4643, 4645, 4647, 4649, 4651, 4656, 4663, 4669, 4674, + 4681, 4683, 4685, 4687, 4689, 4691, 4693, 4694, 4696, 4700, + 4702, 4703, 4711, 4715, 4717, 4720, 4724, 4727, 4728, 4731, + 4732, 4735, 4740, 4746, 4750, 4756, 4758, 4759, 4762, 4763, + 4766, 4770, 4774, 4778, 4780, 4782, 4784, 4787, 4791, 4794, + 4797, 4800, 4803, 4807, 4812, 4816, 4818, 4820, 4822, 4824, + 4825, 4827, 4831, 4833, 4837, 4840, 4850, 4863, 4875, 4888, + 4903, 4907, 4912, 4917, 4918, 4926, 4937, 4947, 4950, 4954, + 4955, 4960, 4962, 4964, 4966, 4968, 4970, 4972, 4974, 4976, + 4978, 4980, 4982, 4984, 4986, 4988, 4990, 4992, 4994, 4996, + 4998, 5000, 5002, 5004, 5006, 5008, 5010, 5012, 5014, 5016, + 5018, 5020, 5022, 5024, 5026, 5028, 5030, 5032, 5034, 5036, + 5038, 5040, 5042, 5044, 5046, 5048, 5050, 5052, 5054, 5056, + 5058, 5060, 5062, 5064, 5066, 5068, 5070, 5072, 5074, 5076, + 5078, 5080, 5082, 5084, 5086, 5088, 5090, 5092, 5094, 5096, + 5098, 5100, 5102, 5104, 5106, 5108, 5110, 5112, 5114, 5116, + 5118, 5120, 5122, 5124, 5126, 5128, 5130, 5132, 5134, 5136, + 5138, 5140, 5142, 5144, 5146, 5148, 5150, 5152, 5154, 5156, + 5158, 5160, 5162, 5164, 5166, 5168, 5170, 5172, 5174, 5176, + 5178, 5180, 5182, 5184, 5186, 5188, 5190, 5192, 5194, 5196, + 5198, 5200, 5202, 5204, 5206, 5208, 5210, 5212, 5214, 5216, + 5218, 5220, 5222, 5224, 5226, 5228, 5230, 5232, 5234, 5236, + 5238, 5240, 5242, 5244, 5246, 5248, 5250, 5252, 5254, 5256, + 5258, 5260, 5262, 5264, 5266, 5268, 5270, 5272, 5274, 5276, + 5278, 5280, 5282, 5284, 5286, 5288, 5290, 5292, 5294, 5296, + 5298, 5300, 5302, 5304, 5306, 5308, 5310, 5312, 5314, 5316, + 5318, 5320, 5322, 5324, 5326, 5328, 5330, 5332, 5334, 5336, + 5338, 5340, 5342, 5344, 5346, 5348, 5350, 5352, 5354, 5356, + 5358, 5360, 5362, 5364, 5366, 5368, 5370, 5372, 5374, 5376, + 5378, 5380, 5382, 5384, 5386, 5388, 5390, 5392, 5394, 5396, + 5398, 5400, 5402, 5404, 5406, 5408, 5410, 5412, 5414, 5416, + 5418, 5420, 5422, 5424, 5426, 5428, 5430, 5432, 5434, 5436, + 5438, 5440, 5442, 5444, 5446, 5448, 5450, 5452, 5454, 5456, + 5458, 5460, 5462, 5464, 5466, 5468, 5470, 5472, 5474, 5476, + 5478, 5480, 5482, 5484, 5486, 5488, 5490, 5492, 5494, 5496, + 5498, 5500, 5502, 5504, 5506, 5508, 5510, 5512, 5514, 5516, + 5518, 5520, 5522, 5524, 5526, 5528, 5530, 5532, 5534, 5536, + 5538, 5540, 5542, 5544, 5546, 5548, 5550, 5552, 5554, 5556, + 5558, 5560, 5562, 5564, 5566, 5568, 5570, 5572, 5574, 5576, + 5578, 5580, 5582, 5584, 5586, 5588, 5590, 5592, 5594, 5596, + 5598, 5600, 5602, 5604, 5606, 5608, 5610, 5612, 5614, 5616, + 5618, 5620, 5622, 5624, 5626, 5628, 5630, 5632, 5634, 5636, + 5638, 5640, 5642, 5644, 5646, 5648, 5650, 5652, 5654, 5656, + 5658, 5660, 5662, 5664, 5666, 5668, 5670, 5672, 5674, 5676, + 5678, 5680, 5682, 5684, 5686, 5688, 5690, 5692, 5694, 5696, + 5698, 5700, 5702, 5704, 5706, 5708, 5710, 5712, 5714, 5716, + 5718, 5720, 5722, 5724, 5726, 5728, 5730, 5732, 5734, 5736, + 5738, 5740, 5742, 5744, 5746, 5748, 5750, 5752, 5754, 5756, + 5758, 5760, 5762, 5764, 5766, 5768, 5770, 5772, 5774, 5776, + 5778, 5780, 5782, 5784, 5786, 5788, 5790, 5792, 5794, 5796, + 5798, 5800, 5802, 5804, 5806, 5808, 5810, 5812, 5814, 5816, + 5818, 5820, 5822, 5824, 5826, 5828, 5830, 5832, 5834, 5836, + 5838, 5840, 5842, 5844, 5846, 5848, 5850, 5852, 5854, 5856, + 5858, 5860, 5862, 5864, 5866, 5868, 5870, 5872, 5874, 5876, + 5878, 5880, 5882, 5884, 5886, 5888, 5890, 5892, 5894, 5896, + 5898, 5900, 5902, 5904, 5906, 5908, 5910, 5912, 5914, 5916, + 5918, 5920, 5922, 5924, 5926, 5928, 5930, 5932, 5934, 5936, + 5938, 5940, 5942, 5944, 5946, 5948, 5950, 5952, 5954, 5956, + 5958, 5960, 5962, 5964, 5966, 5968, 5970, 5972, 5974, 5976, + 5978, 5980, 5982, 5984, 5986, 5988, 5990, 5992, 5994, 5996, + 5998, 6000, 6002, 6004, 6006, 6008, 6010, 6012, 6014, 6016, + 6018, 6020, 6022, 6024, 6026, 6028, 6030, 6032, 6034, 6036, + 6038, 6040, 6042, 6044, 6046, 6048, 6050, 6052, 6054, 6056, + 6058, 6060, 6062, 6064, 6066, 6068, 6070, 6072, 6074, 6076, + 6078, 6080, 6082, 6084, 6086, 6088, 6090, 6092, 6094, 6096, + 6098, 6100, 6102, 6104, 6106, 6108, 6110, 6112, 6114, 6116, + 6118, 6120, 6122, 6124, 6126, 6128, 6130, 6132, 6134, 6136, + 6138, 6140, 6142, 6144, 6146, 6148, 6150, 6152, 6154, 6156, + 6158, 6160, 6162, 6164, 6166, 6168, 6170, 6172, 6174 }; /* YYRHS -- A `-1'-separated list of the rules' RHS. */ static const yytype_int16 yyrhs[] = { - 512, 0, -1, 513, -1, 513, 503, 514, -1, 514, - -1, 893, -1, 568, -1, 515, -1, 927, -1, 928, - -1, 940, -1, 894, -1, 637, -1, 943, -1, 633, - -1, 883, -1, 562, -1, 578, -1, 558, -1, 526, - -1, 923, -1, 929, -1, 626, -1, 564, -1, 898, - -1, 896, -1, 897, -1, 886, -1, 537, -1, 915, - -1, 561, -1, 880, -1, 535, -1, 654, -1, 575, - -1, 636, -1, 577, -1, 918, -1, 932, -1, 909, - -1, 935, -1, 941, -1, -1, 32, 405, 741, 523, - -1, 32, 405, 186, 148, 741, 523, -1, 32, 197, - 527, 523, -1, 32, 197, 186, 148, 527, 523, -1, - 32, 370, 527, 523, -1, 32, 370, 186, 148, 527, - 523, -1, 32, 454, 527, 523, -1, 32, 454, 186, - 148, 527, 523, -1, 518, -1, 516, 518, -1, 375, - 113, 784, -1, 133, 113, -1, 347, -1, 347, 570, - 571, -1, 375, 572, -1, 375, 171, 625, -1, 522, - -1, 519, 504, 522, -1, 26, 605, -1, 26, 186, - 267, 148, 605, -1, 26, 80, 605, -1, 26, 80, - 186, 267, 148, 605, -1, 32, 536, 528, 517, -1, - 32, 536, 528, 133, 267, 272, -1, 32, 536, 528, - 375, 267, 272, -1, 32, 536, 528, 375, 391, 574, - -1, 32, 536, 528, 375, 593, -1, 32, 536, 528, - 345, 593, -1, 32, 536, 528, 375, 394, 528, -1, - 32, 536, 528, 26, 171, 625, 40, 185, 581, -1, - 32, 536, 528, 516, -1, 32, 536, 528, 133, 185, - -1, 32, 536, 528, 133, 185, 186, 148, -1, 133, - 536, 186, 148, 528, 630, -1, 133, 536, 528, 630, - -1, 32, 536, 528, 525, 427, 754, 750, 521, -1, - 32, 536, 528, 524, -1, 26, 595, -1, 32, 91, - 868, 579, -1, 445, 91, 868, -1, 133, 91, 186, - 148, 868, 630, -1, 133, 91, 868, 630, -1, 375, - 239, -1, 375, 436, -1, 375, 593, -1, 345, 593, - -1, 524, -1, 442, 784, -1, -1, 589, -1, 375, - 589, -1, 26, 589, -1, 133, 603, -1, 520, -1, - 523, 504, 520, -1, 286, 500, 519, 501, -1, 375, - 105, -1, 375, -1, -1, 109, 868, -1, 109, 316, - 868, -1, 109, 30, -1, 109, 316, 30, -1, 529, - -1, 528, 531, -1, 3, -1, 946, -1, 947, -1, - 528, -1, 5, -1, 5, -1, 532, -1, 531, 532, - -1, 502, 533, -1, 534, -1, 3, -1, 950, -1, - 946, -1, 952, -1, 32, 361, 868, 341, 416, 868, - -1, 32, 405, 741, 341, 416, 868, -1, 32, 405, - 186, 148, 741, 341, 416, 868, -1, 32, 370, 527, - 341, 416, 868, -1, 32, 370, 186, 148, 527, 341, - 416, 868, -1, 32, 454, 527, 341, 416, 868, -1, - 32, 454, 186, 148, 527, 341, 416, 868, -1, 32, - 197, 527, 341, 416, 868, -1, 32, 197, 186, 148, - 527, 341, 416, 868, -1, 32, 405, 741, 341, 536, - 868, 416, 868, -1, 32, 405, 186, 148, 741, 341, - 536, 868, 416, 868, -1, 32, 405, 741, 341, 91, - 868, 416, 868, -1, 32, 405, 186, 148, 741, 341, - 91, 868, 416, 868, -1, 80, -1, -1, 542, 207, - 545, 214, 539, 540, 538, 546, 548, -1, 654, -1, - 295, 549, 447, 654, -1, 500, 553, 501, 654, -1, - 500, 553, 501, 295, 549, 447, 654, -1, 113, 448, - -1, 527, -1, 527, 40, 528, -1, 59, 258, -1, - 59, 311, -1, -1, 500, 556, 501, 747, -1, 282, - 91, 868, -1, -1, 666, -1, -1, 528, 849, -1, - 557, 489, 784, -1, 500, 550, 501, 489, 784, -1, - 287, 343, -1, 287, 187, -1, -1, 282, 89, 541, - 129, 439, 375, 555, 747, -1, 282, 89, 541, 129, - 268, -1, -1, 528, 551, 552, 682, 683, -1, 792, - 551, 552, 682, 683, -1, 500, 784, 501, 551, 552, - 682, 683, -1, 349, 855, -1, -1, 441, -1, 404, - -1, 557, -1, 550, 504, 557, -1, 78, 875, -1, - -1, 875, -1, -1, 543, -1, 553, 504, 543, -1, - 544, -1, 554, 504, 544, -1, 554, -1, 554, 504, - -1, 547, -1, 556, 504, 547, -1, 528, 849, -1, - 98, 427, 527, 40, 140, 655, -1, 98, 427, 527, - 40, 140, 500, 559, 501, -1, 98, 427, 527, 40, - 754, -1, 560, -1, -1, 530, -1, 560, 504, 530, - -1, 313, 528, -1, 313, 528, 489, 914, -1, 313, - 528, 500, 829, 501, -1, 98, 624, 370, 527, 563, - -1, 98, 624, 370, 186, 267, 148, 527, 563, -1, - 98, 287, 343, 624, 370, 527, 563, -1, 569, -1, - -1, 147, 868, 567, -1, 98, 624, 405, 945, 40, - 147, 868, 567, 944, -1, 98, 624, 405, 186, 267, - 148, 945, 40, 147, 868, 567, 944, -1, 784, -1, - 878, 13, 784, -1, 565, -1, 566, 504, 565, -1, - 500, 566, 501, -1, -1, 32, 370, 527, 569, -1, - 32, 370, 186, 148, 527, 569, -1, 572, -1, 569, - 572, -1, 462, -1, 486, -1, -1, 4, -1, 491, - 4, -1, 492, 4, -1, 574, -1, 40, 756, -1, - 60, 571, -1, 104, -1, 265, 104, -1, 196, 573, - 571, -1, 245, 571, -1, 253, 571, -1, 265, 245, - -1, 265, 253, -1, 296, 59, 875, -1, 370, 258, - 875, -1, 389, 570, 571, -1, 347, -1, 347, 570, - 571, -1, 59, -1, -1, 871, -1, 491, 871, -1, - 492, 871, -1, 22, 576, -1, 52, 576, -1, 389, - 576, -1, 84, 576, -1, 139, 576, -1, 354, 576, - -1, 465, -1, 418, -1, -1, 440, 527, -1, 98, - 624, 405, 527, 500, 611, 501, 600, 592, -1, 98, - 624, 405, 186, 267, 148, 527, 500, 611, 501, 600, - 592, -1, 98, 287, 343, 624, 405, 527, 500, 611, - 501, 600, 592, -1, -1, 579, 604, -1, 619, -1, - 952, -1, 821, -1, 571, -1, 530, -1, 266, -1, - 500, 569, 501, -1, -1, 530, -1, 265, 25, -1, - 348, -1, 63, -1, 375, 272, -1, 375, 113, -1, - 91, 868, 585, -1, 585, -1, 599, -1, 78, 875, - -1, 267, 272, -1, 272, -1, 433, 610, -1, 319, - 221, 610, -1, 72, 500, 784, 501, 594, -1, 442, - 86, 868, -1, 113, 785, -1, 335, 527, 613, 622, - 591, -1, 456, -1, 395, -1, 586, -1, -1, 171, - 625, 40, 185, 581, -1, 171, 625, 40, 500, 784, - 501, 587, -1, 40, 500, 784, 501, 587, -1, 603, - 582, -1, 282, 439, 583, -1, 590, -1, 615, -1, - 590, 615, -1, 615, 590, -1, -1, 282, 84, 133, - -1, 282, 84, 118, 357, -1, 282, 84, 318, 357, - -1, -1, 500, 597, 501, -1, 265, 199, -1, -1, - 91, 868, 620, -1, 620, -1, 83, -1, 92, -1, - 114, -1, 185, -1, 198, -1, 391, -1, 394, -1, - 30, -1, 616, -1, 597, 504, 616, -1, 442, 197, - 607, -1, 115, -1, 267, 115, -1, 201, 116, -1, - 201, 189, -1, 462, 593, -1, 462, 280, -1, 464, - 280, -1, -1, 500, 606, 501, -1, 602, 195, 596, - -1, 602, 145, 596, -1, -1, 534, -1, 267, 115, - -1, 115, -1, 201, 189, -1, 201, 116, -1, 267, - 444, -1, 265, 199, -1, 528, 754, 614, -1, 528, - 753, 588, 614, -1, 609, -1, 606, 504, 609, -1, - 528, -1, 605, -1, 623, -1, 595, -1, 534, 489, - 580, -1, 534, -1, 462, 601, -1, -1, 621, -1, - 621, 504, -1, -1, 528, -1, 500, 617, 501, -1, - -1, 614, 584, -1, -1, 282, 118, 583, -1, 534, - 489, 580, -1, 534, -1, 534, 502, 534, 489, 580, - -1, 534, 502, 534, -1, 612, -1, 617, 504, 612, - -1, 617, -1, 617, 504, -1, 754, -1, 872, 876, - 495, 427, -1, 376, 872, 876, 495, 427, -1, 72, - 500, 784, 501, 579, -1, 433, 500, 618, 501, 610, - 579, -1, 433, 598, 579, -1, 319, 221, 500, 618, - 501, 610, 579, -1, 319, 221, 598, 579, -1, 164, - 221, 500, 618, 501, 335, 527, 613, 622, 591, 579, - -1, 608, -1, 621, 504, 608, -1, 243, 168, -1, - 243, 300, -1, 243, 381, -1, -1, 231, 527, 602, - -1, 411, -1, 409, -1, 235, 411, -1, 235, 409, - -1, 173, 411, -1, 173, 409, -1, 436, -1, -1, - 33, -1, 59, 113, -1, 133, 627, 186, 148, 629, - 630, -1, 133, 627, 629, 630, -1, 133, 628, 186, - 148, 865, 630, -1, 133, 628, 865, 630, -1, 133, - 631, 868, 282, 875, 630, -1, 133, 631, 186, 148, - 868, 282, 875, 630, -1, 133, 427, 632, 630, -1, - 133, 427, 186, 148, 632, 630, -1, 405, -1, 370, - -1, 169, -1, 240, -1, 240, 405, -1, 454, -1, - 244, 454, -1, 197, -1, 164, 405, -1, 79, -1, - 95, -1, 361, -1, 391, -1, 412, 364, 299, -1, - 412, 364, 125, -1, 412, 364, 410, -1, 412, 364, - 88, -1, 24, 246, -1, 142, 421, -1, 152, -1, - 164, 105, 466, -1, 325, -1, 373, -1, 875, -1, - 629, 504, 875, -1, 63, -1, 348, -1, -1, 310, - -1, 358, -1, 421, -1, 754, -1, 632, 504, 754, - -1, 98, 624, 634, 527, 635, 40, 405, 654, -1, - 98, 624, 634, 186, 267, 148, 527, 635, 40, 405, - 654, -1, 98, 287, 343, 624, 634, 527, 635, 40, - 405, 654, -1, 98, 624, 634, 527, 635, 40, 784, - -1, 98, 624, 634, 186, 267, 148, 527, 635, 40, - 784, -1, 98, 287, 343, 624, 634, 527, 635, 40, - 784, -1, 169, -1, 240, -1, 500, 501, -1, 500, - 829, 501, -1, 542, 439, 924, 375, 555, 721, 925, - 548, -1, 96, 649, 527, 613, 647, 638, 643, 652, - 639, 570, 644, -1, 96, 500, 654, 501, 416, 643, - 652, 570, 644, -1, 167, -1, 416, -1, 641, 120, - 530, -1, -1, 651, -1, 640, 504, 651, -1, 442, - -1, -1, 40, -1, -1, 324, -1, -1, 648, -1, - 500, 653, 501, -1, 906, -1, 571, -1, 493, -1, - 500, 640, 501, -1, 789, -1, -1, 534, 645, -1, - 462, 280, -1, -1, 648, 650, -1, -1, 55, -1, - -1, 55, -1, 280, -1, 166, -1, 119, 642, 530, - -1, 272, 642, 530, -1, 100, -1, 181, -1, 327, - 642, 530, -1, 141, 642, 530, -1, 163, 327, 617, - -1, 163, 327, 493, -1, 301, 59, 617, -1, 301, - 59, 493, -1, 163, 267, 272, 617, -1, 163, 272, - 617, -1, 137, 530, -1, 906, -1, 530, -1, 392, - -1, 393, -1, 646, -1, 653, 504, 646, -1, 656, - -1, 655, -1, 500, 656, 501, -1, 500, 655, 501, - -1, 659, -1, 657, 679, -1, 657, 678, 712, 685, - -1, 657, 678, 684, 713, -1, 666, 657, -1, 666, - 657, 679, -1, 666, 657, 678, 712, 685, -1, 666, - 657, 678, 684, 713, -1, 659, -1, 655, -1, 368, - 676, 854, -1, -1, 368, 676, 854, 670, 721, 747, - 701, 710, 798, 711, 689, -1, 368, 675, 856, 670, - 721, 747, 701, 710, 798, 711, 689, -1, 167, 722, - 658, 670, 747, 701, 710, 798, 711, 689, -1, 167, - 722, 368, 675, 856, 670, 747, 701, 710, 798, 711, - 689, -1, 720, -1, 405, 741, -1, 657, 432, 673, - 674, 657, -1, 657, 432, 673, 657, -1, 657, 212, - 673, 657, -1, 657, 143, 673, 657, -1, 661, 724, - 442, 856, -1, 661, 724, 442, 856, 176, 59, 867, - -1, 661, 724, 176, 59, 867, -1, 661, 724, 282, - 665, -1, 661, 724, 282, 665, 176, 59, 867, -1, - 661, 724, 282, 665, 442, 856, -1, 661, 724, 282, - 665, 442, 856, 176, 59, 867, -1, 662, 724, 282, - 856, 214, 258, 868, 660, 867, -1, 662, 724, 282, - 856, -1, 447, -1, 448, -1, 305, -1, 307, -1, - 437, -1, 306, -1, 785, -1, 785, 193, 500, 656, - 501, -1, 727, -1, 663, -1, 664, 504, 663, -1, - 664, -1, 664, 504, -1, 462, 667, -1, 486, 667, - -1, 462, 333, 667, -1, 668, -1, 667, 504, 668, - -1, 868, 877, 40, 669, 500, 882, 501, -1, 244, - -1, 267, 244, -1, -1, 214, 671, -1, -1, 411, - 672, 527, -1, 409, 672, 527, -1, 235, 411, 672, - 527, -1, 235, 409, 672, 527, -1, 173, 411, 672, - 527, -1, 173, 409, 672, 527, -1, 436, 672, 527, - -1, 405, 527, -1, 527, -1, 405, -1, -1, 30, - -1, 128, -1, -1, 59, 258, -1, 128, -1, 128, - 282, 500, 827, 501, -1, 30, -1, -1, 187, 274, - -1, 346, 274, -1, -1, 679, -1, -1, 288, 59, - 680, -1, 288, 59, 30, 682, 683, -1, 681, -1, - 680, 504, 681, -1, 784, 442, 821, 683, -1, 784, - 682, 683, -1, 41, -1, 122, -1, -1, 485, 159, - -1, 485, 225, -1, -1, 686, 687, -1, 687, 686, - -1, 686, -1, 687, -1, 684, -1, -1, 232, 695, - -1, 232, 695, 504, 696, -1, 157, 700, 697, 699, - 283, -1, 157, 700, 699, 283, -1, 279, 696, -1, - 279, 697, 699, -1, 4, 495, -1, 9, 495, -1, - 4, 304, -1, 9, 304, -1, 9, -1, 9, 357, - -1, 442, 359, 691, -1, -1, 528, -1, -1, 690, - 500, 688, 501, 694, -1, 688, -1, 688, 500, 528, - 501, -1, 688, 500, 528, 504, 9, 501, -1, 407, - 691, -1, 692, -1, -1, 342, 500, 9, 501, -1, - -1, 784, -1, 30, -1, 784, 495, -1, 4, 304, - -1, 9, 304, -1, 784, -1, 786, -1, 491, 698, - -1, 492, 698, -1, 871, -1, 4, -1, 356, -1, - 357, -1, 159, -1, 264, -1, 176, 59, 703, -1, - 176, 59, 30, -1, -1, 704, -1, 702, 504, 704, - -1, 702, -1, 702, 504, -1, 784, -1, 705, -1, - 707, -1, 706, -1, 708, -1, 500, 501, -1, 355, - 500, 827, 501, -1, 101, 500, 827, 501, -1, 177, - 377, 500, 703, 501, -1, 177, -1, 178, -1, 180, - 784, -1, -1, 326, 784, -1, -1, 714, -1, 162, - 329, 283, -1, 712, -1, -1, 715, -1, 714, 715, - -1, 716, 717, 718, -1, 162, 439, -1, 162, 265, - 221, 439, -1, 162, 378, -1, 162, 221, 378, -1, - 277, 864, -1, -1, 271, -1, 382, 238, -1, -1, - 448, 500, 827, 501, -1, 719, 504, 500, 827, 501, - -1, 719, -1, 719, 504, -1, 167, 723, -1, -1, - 724, -1, 722, 504, 724, -1, 722, -1, 722, 504, - -1, 741, 736, 693, -1, 742, 737, 693, -1, 720, - 735, 693, -1, 226, 742, 737, -1, 655, 736, 693, - -1, 226, 655, 736, -1, 734, -1, 500, 734, 501, - 735, -1, 724, 305, 500, 856, 162, 730, 725, 501, - 736, -1, 724, 437, 726, 500, 731, 162, 733, 501, - 736, -1, 176, 59, 866, -1, -1, 194, 274, -1, - 144, 274, -1, -1, 785, 193, 500, 856, 501, -1, - 785, 193, 529, -1, 787, -1, 500, 825, 501, -1, - 728, 193, 500, 856, 501, -1, 728, 193, 529, -1, - 729, -1, 730, 729, -1, 529, -1, 500, 866, 501, - -1, 731, 193, 500, 856, 501, -1, 732, -1, 733, - 732, -1, 500, 734, 501, -1, 724, 99, 219, 724, - -1, 724, 738, 219, 724, 740, -1, 724, 219, 724, - 740, -1, 724, 261, 738, 219, 724, -1, 724, 261, - 219, 724, -1, 724, 42, 738, 219, 724, 740, -1, - 724, 42, 219, 724, 740, -1, 724, 312, 219, 724, - -1, 724, 37, 219, 724, 740, -1, 724, 369, 219, - 724, 740, -1, 40, 529, 500, 866, 501, -1, 40, - 529, -1, 528, 500, 866, 501, -1, 528, -1, 735, - -1, -1, 735, -1, 40, 500, 748, 501, -1, 40, - 529, 500, 748, 501, -1, 528, 500, 748, 501, -1, - -1, 168, 739, -1, 229, 739, -1, 352, 739, -1, - 369, -1, 37, -1, 203, -1, 291, -1, -1, 442, - 500, 866, 501, -1, 282, 784, -1, 527, -1, 527, - 493, -1, 283, 527, -1, 283, 500, 527, 501, -1, - 792, 746, -1, 357, 167, 500, 744, 501, 746, -1, - 792, 745, -1, 743, -1, 744, 504, 743, -1, 40, - 500, 748, 501, -1, -1, 486, 289, -1, -1, 459, - 784, -1, -1, 749, -1, 748, 504, 749, -1, 529, - 754, 750, -1, 78, 875, -1, -1, 528, 754, -1, - 751, 504, 528, 754, -1, 356, -1, 398, -1, 754, - -1, -1, 756, 755, -1, 376, 756, 755, -1, 756, - 39, 498, 871, 499, -1, 376, 756, 39, 498, 871, - 499, -1, 756, 39, -1, 376, 756, 39, -1, 752, - 500, 751, 501, 755, -1, 241, 500, 831, 501, 755, - -1, 432, 500, 751, 501, 755, -1, 755, 498, 499, - -1, 755, 498, 871, 499, -1, -1, 758, -1, 760, - -1, 762, -1, 766, -1, 772, -1, 773, 783, -1, - 773, 500, 871, 501, -1, 760, -1, 763, -1, 767, - -1, 772, -1, 874, 759, -1, 500, 828, 501, -1, - -1, 210, -1, 211, -1, 383, -1, 54, -1, 330, - -1, 160, 761, -1, 132, 315, -1, 111, 759, -1, - 110, 759, -1, 275, 759, -1, 57, -1, 500, 871, - 501, -1, -1, 764, -1, 765, -1, 764, -1, 765, - -1, 56, 771, 500, 827, 501, -1, 56, 771, -1, - 768, -1, 769, -1, 768, -1, 769, -1, 770, 500, - 871, 501, -1, 770, -1, 70, 771, -1, 69, 771, - -1, 449, -1, 260, 70, 771, -1, 260, 69, 771, - -1, 262, 771, -1, 451, -1, -1, 415, 500, 871, - 501, 774, -1, 415, 774, -1, 414, 500, 871, 501, - 774, -1, 414, 774, -1, 213, -1, 486, 414, 483, - -1, 464, 414, 483, -1, -1, 480, -1, 481, -1, - 255, -1, 256, -1, 107, -1, 108, -1, 183, -1, - 184, -1, 251, -1, 252, -1, 365, -1, 366, -1, - 249, -1, 250, -1, 247, -1, 248, -1, 775, -1, - 776, -1, 777, -1, 778, -1, 779, -1, 780, -1, - 781, -1, 782, -1, 775, 416, 776, -1, 777, 416, - 778, -1, 777, 416, 779, -1, 777, 416, 780, -1, - 778, 416, 779, -1, 778, 416, 780, -1, 779, 416, - 780, -1, -1, 786, -1, 784, 11, 754, -1, 784, - 78, 875, -1, 784, 46, 414, 483, 784, -1, 491, - 784, -1, 492, 784, -1, 784, 491, 784, -1, 784, - 492, 784, -1, 784, 493, 784, -1, 784, 494, 784, - -1, 784, 15, 784, -1, 784, 495, 784, -1, 784, - 496, 784, -1, 784, 16, 784, -1, 784, 487, 784, - -1, 784, 488, 784, -1, 784, 489, 784, -1, 784, - 19, 784, -1, 784, 20, 784, -1, 784, 21, 784, - -1, 784, 820, 784, -1, 820, 784, -1, 784, 820, - -1, 784, 36, 784, -1, 784, 287, 784, -1, 267, - 784, -1, 484, 784, -1, 784, 172, 784, -1, 784, - 231, 784, -1, 784, 231, 784, 141, 784, -1, 784, - 484, 231, 784, -1, 784, 484, 231, 784, 141, 784, - -1, 784, 188, 784, -1, 784, 188, 784, 141, 784, - -1, 784, 484, 188, 784, -1, 784, 484, 188, 784, - 141, 784, -1, 784, 380, 416, 784, -1, 784, 380, - 416, 784, 141, 784, -1, 784, 484, 380, 416, 784, - -1, 784, 484, 380, 416, 784, 141, 784, -1, 784, - 216, 272, -1, 784, 217, -1, 784, 216, 267, 272, - -1, 784, 267, 272, -1, 784, 270, -1, 784, 17, - 784, -1, 784, 18, 784, -1, 809, 293, 809, -1, - 784, 216, 423, -1, 784, 216, 267, 423, -1, 784, - 216, 155, -1, 784, 216, 267, 155, -1, 784, 216, - 434, -1, 784, 216, 267, 434, -1, 784, 216, 128, - 167, 784, -1, 784, 216, 267, 128, 167, 784, -1, - 784, 216, 277, 500, 831, 501, -1, 784, 216, 267, - 277, 500, 831, 501, -1, 784, 53, 853, 785, 36, - 784, -1, 784, 484, 53, 853, 785, 36, 784, -1, - 784, 53, 402, 785, 36, 784, -1, 784, 484, 53, - 402, 785, 36, 784, -1, 784, 193, 841, -1, 784, - 484, 193, 841, -1, 784, 822, 817, 655, -1, 784, - 822, 817, 500, 784, 501, -1, 113, -1, 81, 500, - 784, 501, -1, 493, 859, 863, -1, 528, 502, 493, - 859, 863, -1, 786, -1, 785, 11, 754, -1, 491, - 785, -1, 492, 785, -1, 785, 491, 785, -1, 785, - 492, 785, -1, 785, 493, 785, -1, 785, 494, 785, - -1, 785, 15, 785, -1, 785, 495, 785, -1, 785, - 496, 785, -1, 785, 16, 785, -1, 785, 487, 785, - -1, 785, 488, 785, -1, 785, 489, 785, -1, 785, - 19, 785, -1, 785, 20, 785, -1, 785, 21, 785, - -1, 785, 820, 785, -1, 820, 785, -1, 785, 820, - -1, 785, 216, 128, 167, 785, -1, 785, 216, 267, - 128, 167, 785, -1, 785, 216, 277, 500, 831, 501, - -1, 785, 216, 267, 277, 500, 831, 501, -1, 787, - -1, 809, -1, 788, 852, -1, 847, -1, 870, -1, - 505, 9, -1, 506, 534, -1, 498, 828, 499, -1, - 794, -1, 39, 655, -1, 39, 498, 828, 499, -1, - 842, -1, 655, -1, 655, 531, -1, 148, 655, -1, - 709, 500, 827, 501, -1, 507, -1, 10, -1, 500, - 784, 501, -1, 789, -1, 241, 508, 816, 509, -1, - 791, -1, 508, 812, 509, -1, 869, 500, 501, -1, - 869, 500, 829, 678, 677, 501, -1, 869, 500, 450, - 830, 678, 677, 501, -1, 869, 500, 829, 504, 450, - 830, 678, 677, 501, -1, 869, 500, 30, 829, 678, - 677, 501, -1, 869, 500, 128, 829, 678, 677, 501, - -1, 790, 795, 796, 797, 801, -1, 793, -1, 790, - -1, 793, -1, 79, 162, 500, 784, 501, -1, 66, - 500, 784, 40, 754, 501, -1, 426, 500, 784, 40, - 754, 501, -1, 154, 500, 832, 501, -1, 294, 500, - 834, 501, -1, 311, 500, 836, 501, -1, 400, 500, - 837, 501, -1, 420, 500, 784, 40, 754, 501, -1, - 422, 500, 58, 840, 501, -1, 422, 500, 227, 840, - 501, -1, 422, 500, 417, 840, 501, -1, 422, 500, - 840, 501, -1, 273, 500, 784, 504, 784, 501, -1, - 77, 500, 827, 501, -1, 498, 784, 162, 528, 193, - 784, 499, -1, 498, 784, 162, 528, 193, 786, 186, - 784, 499, -1, 463, 176, 500, 679, 501, -1, -1, - 158, 500, 459, 784, 501, -1, 158, 500, 784, 501, - -1, -1, 151, -1, -1, 461, 799, -1, -1, 800, - -1, 799, 504, 800, -1, 528, 40, 802, -1, 292, - 802, -1, 292, 528, -1, -1, 500, 803, 804, 678, - 805, 501, -1, 528, -1, -1, 301, 59, 826, -1, - -1, 328, 806, -1, 357, 806, -1, -1, 807, -1, - 53, 807, 36, 807, -1, 429, 314, -1, 429, 161, - -1, 102, 356, -1, 784, 314, -1, 784, 161, -1, - 356, 500, 827, 501, -1, 356, 500, 501, -1, 808, - -1, 500, 826, 504, 784, 501, -1, 529, 510, 784, - -1, 810, -1, 811, 504, 810, -1, 811, -1, 811, - 504, -1, 784, 510, 784, -1, 813, -1, 814, 504, - 813, -1, 814, -1, 814, 504, -1, 815, -1, -1, - 38, -1, 385, -1, 30, -1, 8, -1, 819, -1, - 491, -1, 492, -1, 493, -1, 494, -1, 15, -1, - 495, -1, 496, -1, 16, -1, 487, -1, 488, -1, - 489, -1, 19, -1, 20, -1, 21, -1, 8, -1, - 284, 500, 823, 501, -1, 818, -1, 284, 500, 823, - 501, -1, 818, -1, 284, 500, 823, 501, -1, 231, - -1, 484, 231, -1, 172, -1, 484, 172, -1, 188, - -1, 484, 188, -1, 818, -1, 528, 502, 823, -1, - 786, -1, 824, 504, 786, -1, 824, -1, 824, 504, - -1, 784, -1, 826, 504, 784, -1, 826, -1, 826, - 504, -1, 827, -1, -1, 830, -1, 829, 504, 830, - -1, 784, -1, 878, 13, 784, -1, 878, 14, 784, - -1, 754, -1, 831, 504, 754, -1, 833, 167, 784, - -1, -1, 3, -1, 775, -1, 776, -1, 777, -1, - 778, -1, 779, -1, 780, -1, 781, -1, 782, -1, - 530, -1, 784, 835, 838, 839, -1, 784, 835, 838, - -1, 308, 784, -1, 785, 193, 785, -1, -1, 784, - 838, 839, -1, 784, 839, 838, -1, 784, 838, -1, - 784, 839, -1, 826, -1, -1, 167, 784, -1, 162, - 784, -1, 784, 167, 827, -1, 167, 827, -1, 827, - -1, 655, -1, 500, 827, 501, -1, 65, 846, 843, - 845, 139, -1, 844, -1, 843, 844, -1, 458, 784, - 413, 784, -1, 135, 784, -1, -1, 784, -1, -1, - 528, -1, 528, 531, -1, 498, 784, 499, -1, 498, - 848, 510, 848, 499, -1, 498, 848, 510, 848, 510, - 848, 499, -1, 498, 848, 510, 492, 510, 848, 499, - -1, 784, -1, -1, -1, 849, 532, -1, -1, 500, - 501, -1, 500, 829, 501, -1, 502, 533, 850, -1, - 498, 784, 499, -1, 498, 848, 510, 848, 499, -1, - 498, 848, 510, 848, 510, 848, 499, -1, 498, 848, - 510, 492, 510, 848, 499, -1, -1, 852, 851, -1, - 45, -1, -1, 856, -1, -1, 857, -1, 855, 504, - 857, -1, 855, -1, 855, 504, -1, 784, 40, 879, - -1, 784, 3, -1, 784, -1, 144, 500, 866, 501, - -1, 144, 528, -1, 858, -1, -1, 784, 40, 528, - -1, 860, -1, 861, 504, 860, -1, 861, -1, 861, - 504, -1, 343, 500, 862, 501, -1, 343, 860, -1, - -1, 527, -1, 864, 504, 527, -1, 868, -1, 865, - 504, 868, -1, 865, -1, 865, 504, -1, 866, -1, - 500, 866, 501, -1, 529, -1, 873, -1, 528, 531, - -1, 871, -1, 4, -1, 530, 849, -1, 6, -1, - 7, -1, 869, 530, -1, 869, 500, 829, 678, 677, - 501, 530, -1, 757, 530, -1, 773, 500, 784, 501, - 783, -1, 773, 871, 783, -1, 773, 530, 783, -1, - 423, -1, 155, -1, 272, -1, 9, -1, 3, -1, - 946, -1, 951, -1, 3, -1, 946, -1, 948, -1, - 3, -1, 946, -1, 949, -1, 528, -1, 528, 876, - -1, 502, 533, -1, 876, 502, 533, -1, 500, 866, - 501, -1, -1, 872, -1, 534, -1, 5, -1, 316, - 868, 881, 40, 882, -1, 500, 831, 501, -1, -1, - 654, -1, 537, -1, 636, -1, 637, -1, 923, -1, - 98, 361, 527, 884, -1, 98, 361, 186, 267, 148, - 527, 884, -1, 98, 287, 343, 361, 527, 884, -1, - 884, 885, -1, -1, 578, -1, 886, -1, 562, -1, - 941, -1, 98, 892, 197, 889, 890, 282, 527, 888, - 500, 556, 501, 891, 747, -1, 98, 892, 197, 889, - 186, 267, 148, 607, 282, 527, 888, 500, 556, 501, - 891, 747, -1, 528, -1, 442, 887, -1, -1, 87, - -1, -1, 607, -1, -1, 462, 593, -1, -1, 433, - -1, -1, 32, 405, 741, 375, 361, 868, -1, 32, - 405, 186, 148, 741, 375, 361, 868, -1, 32, 370, - 527, 375, 361, 868, -1, 32, 370, 186, 148, 527, - 375, 361, 868, -1, 32, 454, 527, 375, 361, 868, - -1, 32, 454, 186, 148, 527, 375, 361, 868, -1, - 163, 73, 895, -1, 73, 895, -1, 528, -1, -1, - 150, 106, 530, 644, -1, 150, 106, 528, 416, 530, - 644, -1, 192, 106, 530, -1, 149, 901, -1, 149, - 905, 899, 901, -1, 149, 452, 901, -1, 149, 500, - 904, 501, 901, -1, 452, -1, -1, 906, -1, 571, - -1, -1, 893, -1, 568, -1, 515, -1, 940, -1, - 894, -1, 637, -1, 943, -1, 633, -1, 883, -1, - 562, -1, 578, -1, 558, -1, 526, -1, 923, -1, - 626, -1, 564, -1, 886, -1, 537, -1, 915, -1, - 561, -1, 880, -1, 535, -1, 654, -1, 575, -1, - 636, -1, 918, -1, 932, -1, 909, -1, 935, -1, - 941, -1, 3, -1, 946, -1, 950, -1, 902, -1, - 530, -1, 907, -1, 904, 504, 907, -1, 35, -1, - 34, -1, 423, -1, 155, -1, 282, -1, 903, -1, - 908, 900, -1, 902, -1, 905, -1, 375, 910, -1, - 375, 235, 910, -1, 375, 374, 910, -1, 375, 173, - 910, -1, 911, -1, 938, 167, 102, -1, 414, 483, - 913, -1, 361, 530, -1, 938, 416, 914, -1, 938, - 489, 914, -1, 938, 416, 113, -1, 938, 489, 113, - -1, 906, -1, 571, -1, 530, -1, 3, -1, 773, - 530, 783, -1, 773, 500, 871, 501, 530, -1, 571, - -1, 113, -1, 235, -1, 912, -1, 914, 504, 912, - -1, 234, 916, -1, 208, 916, -1, 163, 208, 916, - -1, 208, 916, 167, 917, -1, 163, 208, 916, 167, - 917, -1, 530, -1, 528, -1, 530, -1, 528, -1, - 443, 920, 922, 899, -1, 443, 920, 922, 899, 527, - 877, -1, 443, 920, 922, 899, 927, -1, 443, 500, - 921, 501, -1, 443, 500, 921, 501, 527, 877, -1, - 905, -1, 452, -1, 166, -1, 168, -1, 3, -1, - 168, -1, -1, 919, -1, 921, 504, 919, -1, 166, - -1, -1, 542, 118, 167, 924, 926, 925, 548, -1, - 424, 672, 924, -1, 741, -1, 741, 528, -1, 741, - 40, 528, -1, 459, 784, -1, -1, 442, 723, -1, - -1, 905, 899, -1, 905, 899, 527, 877, -1, 47, - 930, 530, 931, 644, -1, 124, 930, 3, -1, 124, - 106, 186, 148, 3, -1, 106, -1, -1, 40, 528, - -1, -1, 345, 934, -1, 345, 235, 934, -1, 345, - 374, 934, -1, 345, 173, 934, -1, 938, -1, 30, - -1, 933, -1, 414, 483, -1, 418, 218, 230, -1, - 936, 654, -1, 401, 654, -1, 401, 939, -1, 936, - 939, -1, 936, 414, 483, -1, 936, 418, 218, 230, - -1, 936, 30, 937, -1, 936, -1, 379, -1, 123, - -1, 406, -1, -1, 528, -1, 938, 502, 528, -1, - 528, -1, 939, 502, 528, -1, 61, 790, -1, 98, - 624, 454, 527, 613, 891, 40, 654, 942, -1, 98, - 624, 454, 186, 267, 148, 527, 613, 891, 40, 654, - 942, -1, 98, 287, 343, 624, 454, 527, 613, 891, - 40, 654, 942, -1, 98, 624, 333, 454, 527, 500, - 617, 501, 891, 40, 654, 942, -1, 98, 287, 343, - 624, 333, 454, 527, 500, 617, 501, 891, 40, 654, - 942, -1, 462, 72, 285, -1, 462, 64, 72, 285, - -1, 462, 235, 72, 285, -1, -1, 98, 624, 405, - 945, 40, 654, 944, -1, 98, 624, 405, 186, 267, - 148, 945, 40, 654, 944, -1, 98, 287, 343, 624, - 405, 945, 40, 654, 944, -1, 462, 105, -1, 462, - 265, 105, -1, -1, 527, 613, 600, 592, -1, 22, - -1, 23, -1, 24, -1, 25, -1, 26, -1, 27, - -1, 28, -1, 29, -1, 31, -1, 32, -1, 33, - -1, 43, -1, 44, -1, 46, -1, 47, -1, 48, - -1, 50, -1, 51, -1, 52, -1, 59, -1, 60, - -1, 61, -1, 62, -1, 63, -1, 64, -1, 67, - -1, 68, -1, 71, -1, 73, -1, 74, -1, 75, - -1, 76, -1, 82, -1, 83, -1, 84, -1, 85, - -1, 86, -1, 88, -1, 89, -1, 90, -1, 92, - -1, 93, -1, 94, -1, 95, -1, 96, -1, 97, - -1, 100, -1, 101, -1, 102, -1, 103, -1, 104, - -1, 105, -1, 106, -1, 107, -1, 108, -1, 109, - -1, 112, -1, 114, -1, 116, -1, 117, -1, 118, - -1, 119, -1, 120, -1, 121, -1, 123, -1, 124, - -1, 125, -1, 126, -1, 127, -1, 130, -1, 131, - -1, 132, -1, 133, -1, 134, -1, 136, -1, 137, - -1, 138, -1, 140, -1, 141, -1, 142, -1, 144, - -1, 145, -1, 146, -1, 147, -1, 149, -1, 150, - -1, 151, -1, 152, -1, 153, -1, 156, -1, 158, - -1, 159, -1, 161, -1, 163, -1, 165, -1, 169, - -1, 170, -1, 173, -1, 175, -1, 179, -1, 181, - -1, 182, -1, 183, -1, 184, -1, 185, -1, 186, - -1, 187, -1, 189, -1, 190, -1, 191, -1, 192, - -1, 194, -1, 195, -1, 196, -1, 197, -1, 198, - -1, 199, -1, 200, -1, 202, -1, 205, -1, 206, - -1, 207, -1, 208, -1, 209, -1, 215, -1, 218, - -1, 220, -1, 221, -1, 222, -1, 223, -1, 224, - -1, 225, -1, 228, -1, 230, -1, 233, -1, 234, - -1, 235, -1, 236, -1, 237, -1, 238, -1, 239, - -1, 240, -1, 242, -1, 243, -1, 244, -1, 245, - -1, 246, -1, 247, -1, 248, -1, 249, -1, 250, - -1, 251, -1, 252, -1, 253, -1, 254, -1, 255, - -1, 256, -1, 257, -1, 258, -1, 259, -1, 263, - -1, 264, -1, 265, -1, 268, -1, 269, -1, 271, - -1, 274, -1, 276, -1, 277, -1, 278, -1, 280, - -1, 281, -1, 284, -1, 285, -1, 286, -1, 289, - -1, 292, -1, 295, -1, 296, -1, 297, -1, 298, - -1, 299, -1, 300, -1, 301, -1, 302, -1, 303, - -1, 304, -1, 309, -1, 310, -1, 313, -1, 314, - -1, 316, -1, 317, -1, 318, -1, 320, -1, 321, - -1, 322, -1, 323, -1, 324, -1, 325, -1, 327, - -1, 328, -1, 329, -1, 331, -1, 332, -1, 333, - -1, 334, -1, 336, -1, 337, -1, 338, -1, 339, - -1, 340, -1, 341, -1, 342, -1, 343, -1, 344, - -1, 345, -1, 346, -1, 347, -1, 348, -1, 350, - -1, 351, -1, 353, -1, 354, -1, 355, -1, 357, - -1, 358, -1, 359, -1, 360, -1, 361, -1, 362, - -1, 363, -1, 364, -1, 365, -1, 366, -1, 367, - -1, 370, -1, 371, -1, 372, -1, 373, -1, 374, - -1, 375, -1, 377, -1, 378, -1, 379, -1, 381, - -1, 382, -1, 384, -1, 386, -1, 387, -1, 388, - -1, 389, -1, 390, -1, 391, -1, 392, -1, 393, - -1, 394, -1, 395, -1, 396, -1, 397, -1, 399, - -1, 401, -1, 403, -1, 404, -1, 406, -1, 408, - -1, 409, -1, 410, -1, 411, -1, 412, -1, 418, - -1, 419, -1, 421, -1, 424, -1, 425, -1, 427, - -1, 428, -1, 429, -1, 430, -1, 431, -1, 434, - -1, 435, -1, 436, -1, 438, -1, 439, -1, 440, - -1, 441, -1, 443, -1, 444, -1, 445, -1, 446, - -1, 447, -1, 451, -1, 453, -1, 454, -1, 455, - -1, 456, -1, 457, -1, 460, -1, 463, -1, 464, - -1, 465, -1, 466, -1, 467, -1, 468, -1, 480, - -1, 481, -1, 482, -1, 483, -1, 53, -1, 54, - -1, 56, -1, 57, -1, 69, -1, 70, -1, 77, - -1, 81, -1, 110, -1, 111, -1, 148, -1, 154, - -1, 160, -1, 171, -1, 177, -1, 178, -1, 204, - -1, 210, -1, 211, -1, 213, -1, 241, -1, 260, - -1, 262, -1, 266, -1, 273, -1, 275, -1, 290, - -1, 294, -1, 311, -1, 315, -1, 330, -1, 356, - -1, 376, -1, 383, -1, 398, -1, 400, -1, 414, - -1, 415, -1, 420, -1, 422, -1, 426, -1, 448, - -1, 449, -1, 469, -1, 470, -1, 471, -1, 472, - -1, 473, -1, 474, -1, 475, -1, 476, -1, 477, - -1, 478, -1, 479, -1, 42, -1, 49, -1, 55, - -1, 79, -1, 87, -1, 99, -1, 166, -1, 168, - -1, 171, -1, 172, -1, 188, -1, 203, -1, 216, - -1, 217, -1, 219, -1, 229, -1, 231, -1, 241, - -1, 261, -1, 270, -1, 291, -1, 293, -1, 312, - -1, 352, -1, 380, -1, 398, -1, 407, -1, 452, - -1, 37, -1, 42, -1, 49, -1, 55, -1, 79, - -1, 81, -1, 87, -1, 99, -1, 166, -1, 168, - -1, 172, -1, 188, -1, 203, -1, 216, -1, 217, - -1, 219, -1, 229, -1, 231, -1, 261, -1, 270, - -1, 291, -1, 293, -1, 312, -1, 352, -1, 369, - -1, 380, -1, 407, -1, 426, -1, 452, -1, 37, - -1, 42, -1, 49, -1, 53, -1, 54, -1, 55, - -1, 56, -1, 57, -1, 70, -1, 69, -1, 77, - -1, 79, -1, 81, -1, 87, -1, 99, -1, 110, - -1, 111, -1, 148, -1, 154, -1, 160, -1, 166, - -1, 168, -1, 171, -1, 172, -1, 177, -1, 178, - -1, 188, -1, 203, -1, 204, -1, 211, -1, 213, - -1, 210, -1, 216, -1, 217, -1, 219, -1, 229, - -1, 231, -1, 241, -1, 260, -1, 261, -1, 262, - -1, 266, -1, 270, -1, 273, -1, 275, -1, 291, - -1, 290, -1, 293, -1, 294, -1, 311, -1, 312, - -1, 315, -1, 330, -1, 352, -1, 356, -1, 369, - -1, 376, -1, 380, -1, 383, -1, 398, -1, 400, - -1, 407, -1, 414, -1, 415, -1, 420, -1, 422, - -1, 426, -1, 448, -1, 449, -1, 452, -1, 469, - -1, 470, -1, 471, -1, 472, -1, 473, -1, 474, - -1, 475, -1, 476, -1, 477, -1, 478, -1, 479, - -1, 37, -1, 42, -1, 49, -1, 55, -1, 79, - -1, 81, -1, 87, -1, 99, -1, 166, -1, 168, - -1, 171, -1, 172, -1, 188, -1, 203, -1, 216, - -1, 217, -1, 219, -1, 229, -1, 231, -1, 241, - -1, 261, -1, 270, -1, 291, -1, 293, -1, 312, - -1, 352, -1, 369, -1, 380, -1, 398, -1, 407, - -1, 426, -1, 452, -1, 30, -1, 34, -1, 35, - -1, 36, -1, 38, -1, 39, -1, 40, -1, 41, - -1, 45, -1, 58, -1, 65, -1, 66, -1, 72, - -1, 78, -1, 80, -1, 91, -1, 98, -1, 113, - -1, 115, -1, 122, -1, 128, -1, 129, -1, 135, - -1, 139, -1, 143, -1, 155, -1, 157, -1, 162, - -1, 164, -1, 167, -1, 174, -1, 176, -1, 180, - -1, 193, -1, 201, -1, 212, -1, 214, -1, 226, - -1, 227, -1, 232, -1, 267, -1, 272, -1, 279, - -1, 282, -1, 283, -1, 287, -1, 288, -1, 305, - -1, 306, -1, 307, -1, 308, -1, 319, -1, 326, - -1, 335, -1, 349, -1, 368, -1, 385, -1, 402, - -1, 405, -1, 413, -1, 416, -1, 417, -1, 423, - -1, 432, -1, 433, -1, 437, -1, 442, -1, 450, - -1, 458, -1, 459, -1, 461, -1, 462, -1 + 515, 0, -1, 516, -1, 516, 506, 517, -1, 517, + -1, 897, -1, 571, -1, 518, -1, 931, -1, 932, + -1, 944, -1, 898, -1, 640, -1, 947, -1, 636, + -1, 887, -1, 565, -1, 581, -1, 561, -1, 529, + -1, 927, -1, 933, -1, 629, -1, 567, -1, 902, + -1, 900, -1, 901, -1, 890, -1, 540, -1, 919, + -1, 564, -1, 884, -1, 538, -1, 657, -1, 578, + -1, 639, -1, 580, -1, 922, -1, 936, -1, 913, + -1, 939, -1, 945, -1, -1, 32, 407, 744, 526, + -1, 32, 407, 187, 148, 744, 526, -1, 32, 198, + 530, 526, -1, 32, 198, 187, 148, 530, 526, -1, + 32, 372, 530, 526, -1, 32, 372, 187, 148, 530, + 526, -1, 32, 457, 530, 526, -1, 32, 457, 187, + 148, 530, 526, -1, 521, -1, 519, 521, -1, 377, + 113, 787, -1, 133, 113, -1, 349, -1, 349, 573, + 574, -1, 377, 575, -1, 377, 171, 628, -1, 525, + -1, 522, 507, 525, -1, 26, 608, -1, 26, 187, + 268, 148, 608, -1, 26, 80, 608, -1, 26, 80, + 187, 268, 148, 608, -1, 32, 539, 531, 520, -1, + 32, 539, 531, 133, 268, 273, -1, 32, 539, 531, + 377, 268, 273, -1, 32, 539, 531, 377, 393, 577, + -1, 32, 539, 531, 377, 596, -1, 32, 539, 531, + 347, 596, -1, 32, 539, 531, 377, 396, 531, -1, + 32, 539, 531, 26, 171, 628, 40, 186, 584, -1, + 32, 539, 531, 519, -1, 32, 539, 531, 133, 186, + -1, 32, 539, 531, 133, 186, 187, 148, -1, 133, + 539, 187, 148, 531, 633, -1, 133, 539, 531, 633, + -1, 32, 539, 531, 528, 430, 757, 753, 524, -1, + 32, 539, 531, 527, -1, 26, 598, -1, 32, 91, + 872, 582, -1, 448, 91, 872, -1, 133, 91, 187, + 148, 872, 633, -1, 133, 91, 872, 633, -1, 377, + 240, -1, 377, 439, -1, 377, 596, -1, 347, 596, + -1, 527, -1, 445, 787, -1, -1, 592, -1, 377, + 592, -1, 26, 592, -1, 133, 606, -1, 523, -1, + 526, 507, 523, -1, 287, 503, 522, 504, -1, 377, + 105, -1, 377, -1, -1, 109, 872, -1, 109, 318, + 872, -1, 109, 30, -1, 109, 318, 30, -1, 532, + -1, 531, 534, -1, 3, -1, 950, -1, 951, -1, + 531, -1, 5, -1, 5, -1, 535, -1, 534, 535, + -1, 505, 536, -1, 537, -1, 3, -1, 954, -1, + 950, -1, 956, -1, 32, 363, 872, 343, 419, 872, + -1, 32, 407, 744, 343, 419, 872, -1, 32, 407, + 187, 148, 744, 343, 419, 872, -1, 32, 372, 530, + 343, 419, 872, -1, 32, 372, 187, 148, 530, 343, + 419, 872, -1, 32, 457, 530, 343, 419, 872, -1, + 32, 457, 187, 148, 530, 343, 419, 872, -1, 32, + 198, 530, 343, 419, 872, -1, 32, 198, 187, 148, + 530, 343, 419, 872, -1, 32, 407, 744, 343, 539, + 872, 419, 872, -1, 32, 407, 187, 148, 744, 343, + 539, 872, 419, 872, -1, 32, 407, 744, 343, 91, + 872, 419, 872, -1, 32, 407, 187, 148, 744, 343, + 91, 872, 419, 872, -1, 80, -1, -1, 545, 208, + 548, 215, 542, 543, 541, 549, 551, -1, 657, -1, + 297, 552, 450, 657, -1, 503, 556, 504, 657, -1, + 503, 556, 504, 297, 552, 450, 657, -1, 113, 451, + -1, 530, -1, 530, 40, 531, -1, 59, 259, -1, + 59, 313, -1, -1, 503, 559, 504, 750, -1, 283, + 91, 872, -1, -1, 669, -1, -1, 531, 853, -1, + 560, 492, 787, -1, 503, 553, 504, 492, 787, -1, + 288, 345, -1, 288, 188, -1, -1, 283, 89, 544, + 129, 442, 377, 558, 750, -1, 283, 89, 544, 129, + 269, -1, -1, 531, 554, 555, 685, 686, -1, 795, + 554, 555, 685, 686, -1, 503, 787, 504, 554, 555, + 685, 686, -1, 351, 859, -1, -1, 444, -1, 406, + -1, 560, -1, 553, 507, 560, -1, 78, 879, -1, + -1, 879, -1, -1, 546, -1, 556, 507, 546, -1, + 547, -1, 557, 507, 547, -1, 557, -1, 557, 507, + -1, 550, -1, 559, 507, 550, -1, 531, 853, -1, + 98, 430, 530, 40, 140, 658, -1, 98, 430, 530, + 40, 140, 503, 562, 504, -1, 98, 430, 530, 40, + 757, -1, 563, -1, -1, 533, -1, 563, 507, 533, + -1, 315, 531, -1, 315, 531, 492, 918, -1, 315, + 531, 503, 833, 504, -1, 98, 627, 372, 530, 566, + -1, 98, 627, 372, 187, 268, 148, 530, 566, -1, + 98, 288, 345, 627, 372, 530, 566, -1, 572, -1, + -1, 147, 872, 570, -1, 98, 627, 407, 949, 40, + 147, 872, 570, 948, -1, 98, 627, 407, 187, 268, + 148, 949, 40, 147, 872, 570, 948, -1, 787, -1, + 882, 13, 787, -1, 568, -1, 569, 507, 568, -1, + 503, 569, 504, -1, -1, 32, 372, 530, 572, -1, + 32, 372, 187, 148, 530, 572, -1, 575, -1, 572, + 575, -1, 465, -1, 489, -1, -1, 4, -1, 494, + 4, -1, 495, 4, -1, 577, -1, 40, 759, -1, + 60, 574, -1, 104, -1, 266, 104, -1, 197, 576, + 574, -1, 246, 574, -1, 254, 574, -1, 266, 246, + -1, 266, 254, -1, 298, 59, 879, -1, 372, 259, + 879, -1, 391, 573, 574, -1, 349, -1, 349, 573, + 574, -1, 59, -1, -1, 875, -1, 494, 875, -1, + 495, 875, -1, 22, 579, -1, 52, 579, -1, 391, + 579, -1, 84, 579, -1, 139, 579, -1, 356, 579, + -1, 468, -1, 421, -1, -1, 443, 530, -1, 98, + 627, 407, 530, 503, 614, 504, 603, 595, -1, 98, + 627, 407, 187, 268, 148, 530, 503, 614, 504, 603, + 595, -1, 98, 288, 345, 627, 407, 530, 503, 614, + 504, 603, 595, -1, -1, 582, 607, -1, 622, -1, + 956, -1, 825, -1, 574, -1, 533, -1, 267, -1, + 503, 572, 504, -1, -1, 533, -1, 266, 25, -1, + 350, -1, 63, -1, 377, 273, -1, 377, 113, -1, + 91, 872, 588, -1, 588, -1, 602, -1, 78, 879, + -1, 268, 273, -1, 273, -1, 436, 613, -1, 321, + 222, 613, -1, 72, 503, 787, 504, 597, -1, 445, + 86, 872, -1, 113, 788, -1, 337, 530, 616, 625, + 594, -1, 459, -1, 397, -1, 589, -1, -1, 171, + 628, 40, 186, 584, -1, 171, 628, 40, 503, 787, + 504, 590, -1, 40, 503, 787, 504, 590, -1, 606, + 585, -1, 283, 442, 586, -1, 593, -1, 618, -1, + 593, 618, -1, 618, 593, -1, -1, 283, 84, 133, + -1, 283, 84, 118, 359, -1, 283, 84, 320, 359, + -1, -1, 503, 600, 504, -1, 266, 200, -1, -1, + 91, 872, 623, -1, 623, -1, 83, -1, 92, -1, + 114, -1, 186, -1, 199, -1, 393, -1, 396, -1, + 30, -1, 619, -1, 600, 507, 619, -1, 445, 198, + 610, -1, 115, -1, 268, 115, -1, 202, 116, -1, + 202, 190, -1, 465, 596, -1, 465, 281, -1, 467, + 281, -1, -1, 503, 609, 504, -1, 605, 196, 599, + -1, 605, 145, 599, -1, -1, 537, -1, 268, 115, + -1, 115, -1, 202, 190, -1, 202, 116, -1, 268, + 447, -1, 266, 200, -1, 531, 757, 617, -1, 531, + 756, 591, 617, -1, 612, -1, 609, 507, 612, -1, + 531, -1, 608, -1, 626, -1, 598, -1, 537, 492, + 583, -1, 537, -1, 465, 604, -1, -1, 624, -1, + 624, 507, -1, -1, 531, -1, 503, 620, 504, -1, + -1, 617, 587, -1, -1, 283, 118, 586, -1, 537, + 492, 583, -1, 537, -1, 537, 505, 537, 492, 583, + -1, 537, 505, 537, -1, 615, -1, 620, 507, 615, + -1, 620, -1, 620, 507, -1, 757, -1, 876, 880, + 498, 430, -1, 378, 876, 880, 498, 430, -1, 72, + 503, 787, 504, 582, -1, 436, 503, 621, 504, 613, + 582, -1, 436, 601, 582, -1, 321, 222, 503, 621, + 504, 613, 582, -1, 321, 222, 601, 582, -1, 164, + 222, 503, 621, 504, 337, 530, 616, 625, 594, 582, + -1, 611, -1, 624, 507, 611, -1, 244, 168, -1, + 244, 302, -1, 244, 383, -1, -1, 232, 530, 605, + -1, 413, -1, 411, -1, 236, 413, -1, 236, 411, + -1, 173, 413, -1, 173, 411, -1, 439, -1, -1, + 33, -1, 59, 113, -1, 133, 630, 187, 148, 632, + 633, -1, 133, 630, 632, 633, -1, 133, 631, 187, + 148, 869, 633, -1, 133, 631, 869, 633, -1, 133, + 634, 872, 283, 879, 633, -1, 133, 634, 187, 148, + 872, 283, 879, 633, -1, 133, 430, 635, 633, -1, + 133, 430, 187, 148, 635, 633, -1, 407, -1, 372, + -1, 169, -1, 241, -1, 241, 407, -1, 457, -1, + 245, 457, -1, 198, -1, 164, 407, -1, 79, -1, + 95, -1, 363, -1, 393, -1, 414, 366, 301, -1, + 414, 366, 125, -1, 414, 366, 412, -1, 414, 366, + 88, -1, 24, 247, -1, 142, 424, -1, 152, -1, + 164, 105, 469, -1, 327, -1, 375, -1, 879, -1, + 632, 507, 879, -1, 63, -1, 350, -1, -1, 312, + -1, 360, -1, 424, -1, 757, -1, 635, 507, 757, + -1, 98, 627, 637, 530, 638, 40, 407, 657, -1, + 98, 627, 637, 187, 268, 148, 530, 638, 40, 407, + 657, -1, 98, 288, 345, 627, 637, 530, 638, 40, + 407, 657, -1, 98, 627, 637, 530, 638, 40, 787, + -1, 98, 627, 637, 187, 268, 148, 530, 638, 40, + 787, -1, 98, 288, 345, 627, 637, 530, 638, 40, + 787, -1, 169, -1, 241, -1, 503, 504, -1, 503, + 833, 504, -1, 545, 442, 928, 377, 558, 724, 929, + 551, -1, 96, 652, 530, 616, 650, 641, 646, 655, + 642, 573, 647, -1, 96, 503, 657, 504, 419, 646, + 655, 573, 647, -1, 167, -1, 419, -1, 644, 120, + 533, -1, -1, 654, -1, 643, 507, 654, -1, 445, + -1, -1, 40, -1, -1, 326, -1, -1, 651, -1, + 503, 656, 504, -1, 910, -1, 574, -1, 496, -1, + 503, 643, 504, -1, 792, -1, -1, 537, 648, -1, + 465, 281, -1, -1, 651, 653, -1, -1, 55, -1, + -1, 55, -1, 281, -1, 166, -1, 119, 645, 533, + -1, 273, 645, 533, -1, 100, -1, 182, -1, 329, + 645, 533, -1, 141, 645, 533, -1, 163, 329, 620, + -1, 163, 329, 496, -1, 303, 59, 620, -1, 303, + 59, 496, -1, 163, 268, 273, 620, -1, 163, 273, + 620, -1, 137, 533, -1, 910, -1, 533, -1, 394, + -1, 395, -1, 649, -1, 656, 507, 649, -1, 659, + -1, 658, -1, 503, 659, 504, -1, 503, 658, 504, + -1, 662, -1, 660, 682, -1, 660, 681, 715, 688, + -1, 660, 681, 687, 716, -1, 669, 660, -1, 669, + 660, 682, -1, 669, 660, 681, 715, 688, -1, 669, + 660, 681, 687, 716, -1, 662, -1, 658, -1, 370, + 679, 858, -1, -1, 370, 679, 858, 673, 724, 750, + 704, 713, 801, 714, 692, -1, 370, 678, 860, 673, + 724, 750, 704, 713, 801, 714, 692, -1, 167, 725, + 661, 673, 750, 704, 713, 801, 714, 692, -1, 167, + 725, 370, 678, 860, 673, 750, 704, 713, 801, 714, + 692, -1, 723, -1, 407, 744, -1, 660, 435, 676, + 677, 660, -1, 660, 435, 676, 660, -1, 660, 213, + 676, 660, -1, 660, 143, 676, 660, -1, 664, 727, + 445, 860, -1, 664, 727, 445, 860, 176, 59, 871, + -1, 664, 727, 176, 59, 871, -1, 664, 727, 283, + 668, -1, 664, 727, 283, 668, 176, 59, 871, -1, + 664, 727, 283, 668, 445, 860, -1, 664, 727, 283, + 668, 445, 860, 176, 59, 871, -1, 665, 727, 283, + 860, 215, 259, 872, 663, 871, -1, 665, 727, 283, + 860, -1, 450, -1, 451, -1, 307, -1, 309, -1, + 440, -1, 308, -1, 788, -1, 788, 194, 503, 659, + 504, -1, 730, -1, 666, -1, 667, 507, 666, -1, + 667, -1, 667, 507, -1, 465, 670, -1, 489, 670, + -1, 465, 335, 670, -1, 671, -1, 670, 507, 671, + -1, 872, 881, 40, 672, 503, 886, 504, -1, 245, + -1, 268, 245, -1, -1, 215, 674, -1, -1, 413, + 675, 530, -1, 411, 675, 530, -1, 236, 413, 675, + 530, -1, 236, 411, 675, 530, -1, 173, 413, 675, + 530, -1, 173, 411, 675, 530, -1, 439, 675, 530, + -1, 407, 530, -1, 530, -1, 407, -1, -1, 30, + -1, 128, -1, -1, 59, 259, -1, 128, -1, 128, + 283, 503, 831, 504, -1, 30, -1, -1, 188, 275, + -1, 348, 275, -1, -1, 682, -1, -1, 289, 59, + 683, -1, 289, 59, 30, 685, 686, -1, 684, -1, + 683, 507, 684, -1, 787, 445, 825, 686, -1, 787, + 685, 686, -1, 41, -1, 122, -1, -1, 488, 159, + -1, 488, 226, -1, -1, 689, 690, -1, 690, 689, + -1, 689, -1, 690, -1, 687, -1, -1, 233, 698, + -1, 233, 698, 507, 699, -1, 157, 703, 700, 702, + 284, -1, 157, 703, 702, 284, -1, 280, 699, -1, + 280, 700, 702, -1, 4, 498, -1, 9, 498, -1, + 4, 306, -1, 9, 306, -1, 9, -1, 9, 359, + -1, 445, 361, 694, -1, -1, 531, -1, -1, 693, + 503, 691, 504, 697, -1, 691, -1, 691, 503, 531, + 504, -1, 691, 503, 531, 507, 9, 504, -1, 409, + 694, -1, 695, -1, -1, 344, 503, 9, 504, -1, + -1, 787, -1, 30, -1, 787, 498, -1, 4, 306, + -1, 9, 306, -1, 787, -1, 789, -1, 494, 701, + -1, 495, 701, -1, 875, -1, 4, -1, 358, -1, + 359, -1, 159, -1, 265, -1, 176, 59, 706, -1, + 176, 59, 30, -1, -1, 707, -1, 705, 507, 707, + -1, 705, -1, 705, 507, -1, 787, -1, 708, -1, + 710, -1, 709, -1, 711, -1, 503, 504, -1, 357, + 503, 831, 504, -1, 101, 503, 831, 504, -1, 177, + 379, 503, 706, 504, -1, 177, -1, 178, -1, 181, + 787, -1, -1, 328, 787, -1, -1, 717, -1, 162, + 331, 284, -1, 715, -1, -1, 718, -1, 717, 718, + -1, 719, 720, 721, -1, 162, 442, -1, 162, 266, + 222, 442, -1, 162, 380, -1, 162, 222, 380, -1, + 278, 868, -1, -1, 272, -1, 384, 239, -1, -1, + 451, 503, 831, 504, -1, 722, 507, 503, 831, 504, + -1, 722, -1, 722, 507, -1, 167, 726, -1, -1, + 727, -1, 725, 507, 727, -1, 725, -1, 725, 507, + -1, 744, 739, 696, -1, 745, 740, 696, -1, 723, + 738, 696, -1, 227, 745, 740, -1, 658, 739, 696, + -1, 227, 658, 739, -1, 737, -1, 503, 737, 504, + 738, -1, 727, 307, 503, 860, 162, 733, 728, 504, + 739, -1, 727, 440, 729, 503, 734, 162, 736, 504, + 739, -1, 176, 59, 870, -1, -1, 195, 275, -1, + 144, 275, -1, -1, 788, 194, 503, 860, 504, -1, + 788, 194, 532, -1, 790, -1, 503, 829, 504, -1, + 731, 194, 503, 860, 504, -1, 731, 194, 532, -1, + 732, -1, 733, 732, -1, 532, -1, 503, 870, 504, + -1, 734, 194, 503, 860, 504, -1, 735, -1, 736, + 735, -1, 503, 737, 504, -1, 727, 99, 220, 727, + -1, 727, 741, 220, 727, 743, -1, 727, 220, 727, + 743, -1, 727, 262, 741, 220, 727, -1, 727, 262, + 220, 727, -1, 727, 42, 741, 220, 727, 743, -1, + 727, 42, 220, 727, 743, -1, 727, 314, 220, 727, + -1, 727, 37, 220, 727, 743, -1, 727, 371, 220, + 727, 743, -1, 40, 532, 503, 870, 504, -1, 40, + 532, -1, 531, 503, 870, 504, -1, 531, -1, 738, + -1, -1, 738, -1, 40, 503, 751, 504, -1, 40, + 532, 503, 751, 504, -1, 531, 503, 751, 504, -1, + -1, 168, 742, -1, 230, 742, -1, 354, 742, -1, + 371, -1, 37, -1, 204, -1, 293, -1, -1, 445, + 503, 870, 504, -1, 283, 787, -1, 530, -1, 530, + 496, -1, 284, 530, -1, 284, 503, 530, 504, -1, + 795, 749, -1, 359, 167, 503, 747, 504, 749, -1, + 795, 748, -1, 746, -1, 747, 507, 746, -1, 40, + 503, 751, 504, -1, -1, 489, 290, -1, -1, 462, + 787, -1, -1, 752, -1, 751, 507, 752, -1, 532, + 757, 753, -1, 78, 879, -1, -1, 531, 757, -1, + 754, 507, 531, 757, -1, 358, -1, 400, -1, 757, + -1, -1, 759, 758, -1, 378, 759, 758, -1, 759, + 39, 501, 875, 502, -1, 378, 759, 39, 501, 875, + 502, -1, 759, 39, -1, 378, 759, 39, -1, 755, + 503, 754, 504, 758, -1, 242, 503, 835, 504, 758, + -1, 435, 503, 754, 504, 758, -1, 758, 501, 502, + -1, 758, 501, 875, 502, -1, -1, 761, -1, 763, + -1, 765, -1, 769, -1, 775, -1, 776, 786, -1, + 776, 503, 875, 504, -1, 763, -1, 766, -1, 770, + -1, 775, -1, 878, 762, -1, 503, 832, 504, -1, + -1, 211, -1, 212, -1, 385, -1, 54, -1, 332, + -1, 160, 764, -1, 132, 317, -1, 111, 762, -1, + 110, 762, -1, 276, 762, -1, 57, -1, 503, 875, + 504, -1, -1, 767, -1, 768, -1, 767, -1, 768, + -1, 56, 774, 503, 831, 504, -1, 56, 774, -1, + 771, -1, 772, -1, 771, -1, 772, -1, 773, 503, + 875, 504, -1, 773, -1, 70, 774, -1, 69, 774, + -1, 452, -1, 261, 70, 774, -1, 261, 69, 774, + -1, 263, 774, -1, 454, -1, -1, 418, 503, 875, + 504, 777, -1, 418, 777, -1, 417, 503, 875, 504, + 777, -1, 417, 777, -1, 214, -1, 489, 417, 486, + -1, 467, 417, 486, -1, -1, 483, -1, 484, -1, + 256, -1, 257, -1, 107, -1, 108, -1, 184, -1, + 185, -1, 252, -1, 253, -1, 367, -1, 368, -1, + 250, -1, 251, -1, 248, -1, 249, -1, 778, -1, + 779, -1, 780, -1, 781, -1, 782, -1, 783, -1, + 784, -1, 785, -1, 778, 419, 779, -1, 780, 419, + 781, -1, 780, 419, 782, -1, 780, 419, 783, -1, + 781, 419, 782, -1, 781, 419, 783, -1, 782, 419, + 783, -1, -1, 789, -1, 787, 11, 757, -1, 787, + 78, 879, -1, 787, 46, 417, 486, 787, -1, 494, + 787, -1, 495, 787, -1, 787, 494, 787, -1, 787, + 495, 787, -1, 787, 496, 787, -1, 787, 497, 787, + -1, 787, 15, 787, -1, 787, 498, 787, -1, 787, + 499, 787, -1, 787, 16, 787, -1, 787, 490, 787, + -1, 787, 491, 787, -1, 787, 492, 787, -1, 787, + 19, 787, -1, 787, 20, 787, -1, 787, 21, 787, + -1, 787, 824, 787, -1, 824, 787, -1, 787, 824, + -1, 787, 36, 787, -1, 787, 288, 787, -1, 268, + 787, -1, 487, 787, -1, 787, 172, 787, -1, 787, + 232, 787, -1, 787, 232, 787, 141, 787, -1, 787, + 487, 232, 787, -1, 787, 487, 232, 787, 141, 787, + -1, 787, 189, 787, -1, 787, 189, 787, 141, 787, + -1, 787, 487, 189, 787, -1, 787, 487, 189, 787, + 141, 787, -1, 787, 382, 419, 787, -1, 787, 382, + 419, 787, 141, 787, -1, 787, 487, 382, 419, 787, + -1, 787, 487, 382, 419, 787, 141, 787, -1, 787, + 217, 273, -1, 787, 218, -1, 787, 217, 268, 273, + -1, 787, 268, 273, -1, 787, 271, -1, 787, 17, + 787, -1, 787, 18, 787, -1, 813, 295, 813, -1, + 787, 217, 426, -1, 787, 217, 268, 426, -1, 787, + 217, 155, -1, 787, 217, 268, 155, -1, 787, 217, + 437, -1, 787, 217, 268, 437, -1, 787, 217, 128, + 167, 787, -1, 787, 217, 268, 128, 167, 787, -1, + 787, 217, 278, 503, 835, 504, -1, 787, 217, 268, + 278, 503, 835, 504, -1, 787, 53, 857, 788, 36, + 787, -1, 787, 487, 53, 857, 788, 36, 787, -1, + 787, 53, 404, 788, 36, 787, -1, 787, 487, 53, + 404, 788, 36, 787, -1, 787, 194, 845, -1, 787, + 487, 194, 845, -1, 787, 826, 821, 658, -1, 787, + 826, 821, 503, 787, 504, -1, 113, -1, 81, 503, + 787, 504, -1, 496, 863, 867, -1, 531, 505, 496, + 863, 867, -1, 789, -1, 788, 11, 757, -1, 494, + 788, -1, 495, 788, -1, 788, 494, 788, -1, 788, + 495, 788, -1, 788, 496, 788, -1, 788, 497, 788, + -1, 788, 15, 788, -1, 788, 498, 788, -1, 788, + 499, 788, -1, 788, 16, 788, -1, 788, 490, 788, + -1, 788, 491, 788, -1, 788, 492, 788, -1, 788, + 19, 788, -1, 788, 20, 788, -1, 788, 21, 788, + -1, 788, 824, 788, -1, 824, 788, -1, 788, 824, + -1, 788, 217, 128, 167, 788, -1, 788, 217, 268, + 128, 167, 788, -1, 788, 217, 278, 503, 835, 504, + -1, 788, 217, 268, 278, 503, 835, 504, -1, 790, + -1, 813, -1, 791, 856, -1, 851, -1, 874, -1, + 508, 9, -1, 509, 537, -1, 501, 832, 502, -1, + 797, -1, 39, 658, -1, 39, 501, 832, 502, -1, + 846, -1, 658, -1, 658, 534, -1, 148, 658, -1, + 712, 503, 831, 504, -1, 510, -1, 10, -1, 503, + 787, 504, -1, 792, -1, 242, 511, 820, 512, -1, + 794, -1, 511, 816, 512, -1, 873, 503, 504, -1, + 873, 503, 833, 681, 680, 504, -1, 873, 503, 453, + 834, 681, 680, 504, -1, 873, 503, 833, 507, 453, + 834, 681, 680, 504, -1, 873, 503, 30, 833, 681, + 680, 504, -1, 873, 503, 128, 833, 681, 680, 504, + -1, 793, 798, 799, 800, 804, -1, 796, -1, 793, + -1, 796, -1, 79, 162, 503, 787, 504, -1, 66, + 503, 787, 40, 757, 504, -1, 429, 503, 787, 40, + 757, 504, -1, 154, 503, 836, 504, -1, 296, 503, + 838, 504, -1, 313, 503, 840, 504, -1, 402, 503, + 841, 504, -1, 423, 503, 787, 40, 757, 504, -1, + 425, 503, 58, 844, 504, -1, 425, 503, 228, 844, + 504, -1, 425, 503, 420, 844, 504, -1, 425, 503, + 844, 504, -1, 274, 503, 787, 507, 787, 504, -1, + 77, 503, 831, 504, -1, 501, 787, 162, 531, 194, + 787, 502, -1, 501, 787, 162, 531, 194, 789, 187, + 787, 502, -1, 466, 176, 503, 682, 504, -1, -1, + 158, 503, 462, 787, 504, -1, 158, 503, 787, 504, + -1, -1, 151, -1, -1, 464, 802, -1, -1, 803, + -1, 802, 507, 803, -1, 531, 40, 805, -1, 294, + 805, -1, 294, 531, -1, -1, 503, 806, 807, 681, + 808, 504, -1, 531, -1, -1, 303, 59, 830, -1, + -1, 330, 809, 811, -1, 359, 809, 811, -1, 179, + 809, 811, -1, -1, 810, -1, 53, 810, 36, 810, + -1, 432, 316, -1, 432, 161, -1, 102, 358, -1, + 787, 316, -1, 787, 161, -1, 144, 102, 358, -1, + 144, 176, -1, 144, 416, -1, 144, 266, 291, -1, + -1, 358, 503, 831, 504, -1, 358, 503, 504, -1, + 812, -1, 503, 830, 507, 787, 504, -1, 532, 513, + 787, -1, 814, -1, 815, 507, 814, -1, 815, -1, + 815, 507, -1, 787, 513, 787, -1, 817, -1, 818, + 507, 817, -1, 818, -1, 818, 507, -1, 819, -1, + -1, 38, -1, 387, -1, 30, -1, 8, -1, 823, + -1, 494, -1, 495, -1, 496, -1, 497, -1, 15, + -1, 498, -1, 499, -1, 16, -1, 490, -1, 491, + -1, 492, -1, 19, -1, 20, -1, 21, -1, 8, + -1, 285, 503, 827, 504, -1, 822, -1, 285, 503, + 827, 504, -1, 822, -1, 285, 503, 827, 504, -1, + 232, -1, 487, 232, -1, 172, -1, 487, 172, -1, + 189, -1, 487, 189, -1, 822, -1, 531, 505, 827, + -1, 789, -1, 828, 507, 789, -1, 828, -1, 828, + 507, -1, 787, -1, 830, 507, 787, -1, 830, -1, + 830, 507, -1, 831, -1, -1, 834, -1, 833, 507, + 834, -1, 787, -1, 882, 13, 787, -1, 882, 14, + 787, -1, 757, -1, 835, 507, 757, -1, 837, 167, + 787, -1, -1, 3, -1, 778, -1, 779, -1, 780, + -1, 781, -1, 782, -1, 783, -1, 784, -1, 785, + -1, 533, -1, 787, 839, 842, 843, -1, 787, 839, + 842, -1, 310, 787, -1, 788, 194, 788, -1, -1, + 787, 842, 843, -1, 787, 843, 842, -1, 787, 842, + -1, 787, 843, -1, 830, -1, -1, 167, 787, -1, + 162, 787, -1, 787, 167, 831, -1, 167, 831, -1, + 831, -1, 658, -1, 503, 831, 504, -1, 65, 850, + 847, 849, 139, -1, 848, -1, 847, 848, -1, 461, + 787, 415, 787, -1, 135, 787, -1, -1, 787, -1, + -1, 531, -1, 531, 534, -1, 501, 787, 502, -1, + 501, 852, 513, 852, 502, -1, 501, 852, 513, 852, + 513, 852, 502, -1, 501, 852, 513, 495, 513, 852, + 502, -1, 787, -1, -1, -1, 853, 535, -1, -1, + 503, 504, -1, 503, 833, 504, -1, 505, 536, 854, + -1, 501, 787, 502, -1, 501, 852, 513, 852, 502, + -1, 501, 852, 513, 852, 513, 852, 502, -1, 501, + 852, 513, 495, 513, 852, 502, -1, -1, 856, 855, + -1, 45, -1, -1, 860, -1, -1, 861, -1, 859, + 507, 861, -1, 859, -1, 859, 507, -1, 787, 40, + 883, -1, 787, 3, -1, 787, -1, 144, 503, 870, + 504, -1, 144, 531, -1, 862, -1, -1, 787, 40, + 531, -1, 864, -1, 865, 507, 864, -1, 865, -1, + 865, 507, -1, 345, 503, 866, 504, -1, 345, 864, + -1, -1, 530, -1, 868, 507, 530, -1, 872, -1, + 869, 507, 872, -1, 869, -1, 869, 507, -1, 870, + -1, 503, 870, 504, -1, 532, -1, 877, -1, 531, + 534, -1, 875, -1, 4, -1, 533, 853, -1, 6, + -1, 7, -1, 873, 533, -1, 873, 503, 833, 681, + 680, 504, 533, -1, 760, 533, -1, 776, 503, 787, + 504, 786, -1, 776, 875, 786, -1, 776, 533, 786, + -1, 426, -1, 155, -1, 273, -1, 9, -1, 3, + -1, 950, -1, 955, -1, 3, -1, 950, -1, 952, + -1, 3, -1, 950, -1, 953, -1, 531, -1, 531, + 880, -1, 505, 536, -1, 880, 505, 536, -1, 503, + 870, 504, -1, -1, 876, -1, 537, -1, 5, -1, + 318, 872, 885, 40, 886, -1, 503, 835, 504, -1, + -1, 657, -1, 540, -1, 639, -1, 640, -1, 927, + -1, 98, 363, 530, 888, -1, 98, 363, 187, 268, + 148, 530, 888, -1, 98, 288, 345, 363, 530, 888, + -1, 888, 889, -1, -1, 581, -1, 890, -1, 565, + -1, 945, -1, 98, 896, 198, 893, 894, 283, 530, + 892, 503, 559, 504, 895, 750, -1, 98, 896, 198, + 893, 187, 268, 148, 610, 283, 530, 892, 503, 559, + 504, 895, 750, -1, 531, -1, 445, 891, -1, -1, + 87, -1, -1, 610, -1, -1, 465, 596, -1, -1, + 436, -1, -1, 32, 407, 744, 377, 363, 872, -1, + 32, 407, 187, 148, 744, 377, 363, 872, -1, 32, + 372, 530, 377, 363, 872, -1, 32, 372, 187, 148, + 530, 377, 363, 872, -1, 32, 457, 530, 377, 363, + 872, -1, 32, 457, 187, 148, 530, 377, 363, 872, + -1, 163, 73, 899, -1, 73, 899, -1, 531, -1, + -1, 150, 106, 533, 647, -1, 150, 106, 531, 419, + 533, 647, -1, 193, 106, 533, -1, 149, 905, -1, + 149, 909, 903, 905, -1, 149, 455, 905, -1, 149, + 503, 908, 504, 905, -1, 455, -1, -1, 910, -1, + 574, -1, -1, 897, -1, 571, -1, 518, -1, 944, + -1, 898, -1, 640, -1, 947, -1, 636, -1, 887, + -1, 565, -1, 581, -1, 561, -1, 529, -1, 927, + -1, 629, -1, 567, -1, 890, -1, 540, -1, 919, + -1, 564, -1, 884, -1, 538, -1, 657, -1, 578, + -1, 639, -1, 922, -1, 936, -1, 913, -1, 939, + -1, 945, -1, 3, -1, 950, -1, 954, -1, 906, + -1, 533, -1, 911, -1, 908, 507, 911, -1, 35, + -1, 34, -1, 426, -1, 155, -1, 283, -1, 907, + -1, 912, 904, -1, 906, -1, 909, -1, 377, 914, + -1, 377, 236, 914, -1, 377, 376, 914, -1, 377, + 173, 914, -1, 915, -1, 942, 167, 102, -1, 417, + 486, 917, -1, 363, 533, -1, 942, 419, 918, -1, + 942, 492, 918, -1, 942, 419, 113, -1, 942, 492, + 113, -1, 910, -1, 574, -1, 533, -1, 3, -1, + 776, 533, 786, -1, 776, 503, 875, 504, 533, -1, + 574, -1, 113, -1, 236, -1, 916, -1, 918, 507, + 916, -1, 235, 920, -1, 209, 920, -1, 163, 209, + 920, -1, 209, 920, 167, 921, -1, 163, 209, 920, + 167, 921, -1, 533, -1, 531, -1, 533, -1, 531, + -1, 446, 924, 926, 903, -1, 446, 924, 926, 903, + 530, 881, -1, 446, 924, 926, 903, 931, -1, 446, + 503, 925, 504, -1, 446, 503, 925, 504, 530, 881, + -1, 909, -1, 455, -1, 166, -1, 168, -1, 3, + -1, 168, -1, -1, 923, -1, 925, 507, 923, -1, + 166, -1, -1, 545, 118, 167, 928, 930, 929, 551, + -1, 427, 675, 928, -1, 744, -1, 744, 531, -1, + 744, 40, 531, -1, 462, 787, -1, -1, 445, 726, + -1, -1, 909, 903, -1, 909, 903, 530, 881, -1, + 47, 934, 533, 935, 647, -1, 124, 934, 3, -1, + 124, 106, 187, 148, 3, -1, 106, -1, -1, 40, + 531, -1, -1, 347, 938, -1, 347, 236, 938, -1, + 347, 376, 938, -1, 347, 173, 938, -1, 942, -1, + 30, -1, 937, -1, 417, 486, -1, 421, 219, 231, + -1, 940, 657, -1, 403, 657, -1, 403, 943, -1, + 940, 943, -1, 940, 417, 486, -1, 940, 421, 219, + 231, -1, 940, 30, 941, -1, 940, -1, 381, -1, + 123, -1, 408, -1, -1, 531, -1, 942, 505, 531, + -1, 531, -1, 943, 505, 531, -1, 61, 793, -1, + 98, 627, 457, 530, 616, 895, 40, 657, 946, -1, + 98, 627, 457, 187, 268, 148, 530, 616, 895, 40, + 657, 946, -1, 98, 288, 345, 627, 457, 530, 616, + 895, 40, 657, 946, -1, 98, 627, 335, 457, 530, + 503, 620, 504, 895, 40, 657, 946, -1, 98, 288, + 345, 627, 335, 457, 530, 503, 620, 504, 895, 40, + 657, 946, -1, 465, 72, 286, -1, 465, 64, 72, + 286, -1, 465, 236, 72, 286, -1, -1, 98, 627, + 407, 949, 40, 657, 948, -1, 98, 627, 407, 187, + 268, 148, 949, 40, 657, 948, -1, 98, 288, 345, + 627, 407, 949, 40, 657, 948, -1, 465, 105, -1, + 465, 266, 105, -1, -1, 530, 616, 603, 595, -1, + 22, -1, 23, -1, 24, -1, 25, -1, 26, -1, + 27, -1, 28, -1, 29, -1, 31, -1, 32, -1, + 33, -1, 43, -1, 44, -1, 46, -1, 47, -1, + 48, -1, 50, -1, 51, -1, 52, -1, 59, -1, + 60, -1, 61, -1, 62, -1, 63, -1, 64, -1, + 67, -1, 68, -1, 71, -1, 73, -1, 74, -1, + 75, -1, 76, -1, 82, -1, 83, -1, 84, -1, + 85, -1, 86, -1, 88, -1, 89, -1, 90, -1, + 92, -1, 93, -1, 94, -1, 95, -1, 96, -1, + 97, -1, 100, -1, 101, -1, 102, -1, 103, -1, + 104, -1, 105, -1, 106, -1, 107, -1, 108, -1, + 109, -1, 112, -1, 114, -1, 116, -1, 117, -1, + 118, -1, 119, -1, 120, -1, 121, -1, 123, -1, + 124, -1, 125, -1, 126, -1, 127, -1, 130, -1, + 131, -1, 132, -1, 133, -1, 134, -1, 136, -1, + 137, -1, 138, -1, 140, -1, 141, -1, 142, -1, + 144, -1, 145, -1, 146, -1, 147, -1, 149, -1, + 150, -1, 151, -1, 152, -1, 153, -1, 156, -1, + 158, -1, 159, -1, 161, -1, 163, -1, 165, -1, + 169, -1, 170, -1, 173, -1, 175, -1, 179, -1, + 180, -1, 182, -1, 183, -1, 184, -1, 185, -1, + 186, -1, 187, -1, 188, -1, 190, -1, 191, -1, + 192, -1, 193, -1, 195, -1, 196, -1, 197, -1, + 198, -1, 199, -1, 200, -1, 201, -1, 203, -1, + 206, -1, 207, -1, 208, -1, 209, -1, 210, -1, + 216, -1, 219, -1, 221, -1, 222, -1, 223, -1, + 224, -1, 225, -1, 226, -1, 229, -1, 231, -1, + 234, -1, 235, -1, 236, -1, 237, -1, 238, -1, + 239, -1, 240, -1, 241, -1, 243, -1, 244, -1, + 245, -1, 246, -1, 247, -1, 248, -1, 249, -1, + 250, -1, 251, -1, 252, -1, 253, -1, 254, -1, + 255, -1, 256, -1, 257, -1, 258, -1, 259, -1, + 260, -1, 264, -1, 265, -1, 266, -1, 269, -1, + 270, -1, 272, -1, 275, -1, 277, -1, 278, -1, + 279, -1, 281, -1, 282, -1, 285, -1, 286, -1, + 287, -1, 290, -1, 291, -1, 294, -1, 297, -1, + 298, -1, 299, -1, 300, -1, 301, -1, 302, -1, + 303, -1, 304, -1, 305, -1, 306, -1, 311, -1, + 312, -1, 315, -1, 316, -1, 318, -1, 319, -1, + 320, -1, 322, -1, 323, -1, 324, -1, 325, -1, + 326, -1, 327, -1, 329, -1, 330, -1, 331, -1, + 333, -1, 334, -1, 335, -1, 336, -1, 338, -1, + 339, -1, 340, -1, 341, -1, 342, -1, 343, -1, + 344, -1, 345, -1, 346, -1, 347, -1, 348, -1, + 349, -1, 350, -1, 352, -1, 353, -1, 355, -1, + 356, -1, 357, -1, 359, -1, 360, -1, 361, -1, + 362, -1, 363, -1, 364, -1, 365, -1, 366, -1, + 367, -1, 368, -1, 369, -1, 372, -1, 373, -1, + 374, -1, 375, -1, 376, -1, 377, -1, 379, -1, + 380, -1, 381, -1, 383, -1, 384, -1, 386, -1, + 388, -1, 389, -1, 390, -1, 391, -1, 392, -1, + 393, -1, 394, -1, 395, -1, 396, -1, 397, -1, + 398, -1, 399, -1, 401, -1, 403, -1, 405, -1, + 406, -1, 408, -1, 410, -1, 411, -1, 412, -1, + 413, -1, 414, -1, 416, -1, 421, -1, 422, -1, + 424, -1, 427, -1, 428, -1, 430, -1, 431, -1, + 432, -1, 433, -1, 434, -1, 437, -1, 438, -1, + 439, -1, 441, -1, 442, -1, 443, -1, 444, -1, + 446, -1, 447, -1, 448, -1, 449, -1, 450, -1, + 454, -1, 456, -1, 457, -1, 458, -1, 459, -1, + 460, -1, 463, -1, 466, -1, 467, -1, 468, -1, + 469, -1, 470, -1, 471, -1, 483, -1, 484, -1, + 485, -1, 486, -1, 53, -1, 54, -1, 56, -1, + 57, -1, 69, -1, 70, -1, 77, -1, 81, -1, + 110, -1, 111, -1, 148, -1, 154, -1, 160, -1, + 171, -1, 177, -1, 178, -1, 205, -1, 211, -1, + 212, -1, 214, -1, 242, -1, 261, -1, 263, -1, + 267, -1, 274, -1, 276, -1, 292, -1, 296, -1, + 313, -1, 317, -1, 332, -1, 358, -1, 378, -1, + 385, -1, 400, -1, 402, -1, 417, -1, 418, -1, + 423, -1, 425, -1, 429, -1, 451, -1, 452, -1, + 472, -1, 473, -1, 474, -1, 475, -1, 476, -1, + 477, -1, 478, -1, 479, -1, 480, -1, 481, -1, + 482, -1, 42, -1, 49, -1, 55, -1, 79, -1, + 87, -1, 99, -1, 166, -1, 168, -1, 171, -1, + 172, -1, 189, -1, 204, -1, 217, -1, 218, -1, + 220, -1, 230, -1, 232, -1, 242, -1, 262, -1, + 271, -1, 293, -1, 295, -1, 314, -1, 354, -1, + 382, -1, 400, -1, 409, -1, 455, -1, 37, -1, + 42, -1, 49, -1, 55, -1, 79, -1, 81, -1, + 87, -1, 99, -1, 166, -1, 168, -1, 172, -1, + 189, -1, 204, -1, 217, -1, 218, -1, 220, -1, + 230, -1, 232, -1, 262, -1, 271, -1, 293, -1, + 295, -1, 314, -1, 354, -1, 371, -1, 382, -1, + 409, -1, 429, -1, 455, -1, 37, -1, 42, -1, + 49, -1, 53, -1, 54, -1, 55, -1, 56, -1, + 57, -1, 70, -1, 69, -1, 77, -1, 79, -1, + 81, -1, 87, -1, 99, -1, 110, -1, 111, -1, + 148, -1, 154, -1, 160, -1, 166, -1, 168, -1, + 171, -1, 172, -1, 177, -1, 178, -1, 189, -1, + 204, -1, 205, -1, 212, -1, 214, -1, 211, -1, + 217, -1, 218, -1, 220, -1, 230, -1, 232, -1, + 242, -1, 261, -1, 262, -1, 263, -1, 267, -1, + 271, -1, 274, -1, 276, -1, 293, -1, 292, -1, + 295, -1, 296, -1, 313, -1, 314, -1, 317, -1, + 332, -1, 354, -1, 358, -1, 371, -1, 378, -1, + 382, -1, 385, -1, 400, -1, 402, -1, 409, -1, + 417, -1, 418, -1, 423, -1, 425, -1, 429, -1, + 451, -1, 452, -1, 455, -1, 472, -1, 473, -1, + 474, -1, 475, -1, 476, -1, 477, -1, 478, -1, + 479, -1, 480, -1, 481, -1, 482, -1, 37, -1, + 42, -1, 49, -1, 55, -1, 79, -1, 81, -1, + 87, -1, 99, -1, 166, -1, 168, -1, 171, -1, + 172, -1, 189, -1, 204, -1, 217, -1, 218, -1, + 220, -1, 230, -1, 232, -1, 242, -1, 262, -1, + 271, -1, 293, -1, 295, -1, 314, -1, 354, -1, + 371, -1, 382, -1, 400, -1, 409, -1, 429, -1, + 455, -1, 30, -1, 34, -1, 35, -1, 36, -1, + 38, -1, 39, -1, 40, -1, 41, -1, 45, -1, + 58, -1, 65, -1, 66, -1, 72, -1, 78, -1, + 80, -1, 91, -1, 98, -1, 113, -1, 115, -1, + 122, -1, 128, -1, 129, -1, 135, -1, 139, -1, + 143, -1, 155, -1, 157, -1, 162, -1, 164, -1, + 167, -1, 174, -1, 176, -1, 181, -1, 194, -1, + 202, -1, 213, -1, 215, -1, 227, -1, 228, -1, + 233, -1, 268, -1, 273, -1, 280, -1, 283, -1, + 284, -1, 288, -1, 289, -1, 307, -1, 308, -1, + 309, -1, 310, -1, 321, -1, 328, -1, 337, -1, + 351, -1, 370, -1, 387, -1, 404, -1, 407, -1, + 415, -1, 419, -1, 420, -1, 426, -1, 435, -1, + 436, -1, 440, -1, 445, -1, 453, -1, 461, -1, + 462, -1, 464, -1, 465, -1 }; /* YYRLINE[YYN] -- source line where rule number YYN was defined. */ static const yytype_uint16 yyrline[] = { - 0, 492, 492, 508, 520, 529, 530, 531, 532, 533, - 534, 535, 536, 537, 538, 539, 540, 541, 542, 543, - 544, 545, 546, 547, 548, 549, 550, 551, 552, 553, - 554, 555, 556, 557, 558, 559, 560, 561, 562, 563, - 564, 565, 567, 9, 18, 27, 36, 45, 54, 63, + 0, 493, 493, 509, 521, 530, 531, 532, 533, 534, + 535, 536, 537, 538, 539, 540, 541, 542, 543, 544, + 545, 546, 547, 548, 549, 550, 551, 552, 553, 554, + 555, 556, 557, 558, 559, 560, 561, 562, 563, 564, + 565, 566, 568, 9, 18, 27, 36, 45, 54, 63, 72, 85, 87, 93, 94, 99, 103, 107, 118, 126, 130, 139, 148, 157, 166, 175, 184, 192, 200, 209, 218, 227, 236, 253, 262, 271, 280, 290, 303, 318, @@ -2565,48 +2575,49 @@ static const yytype_uint16 yyrline[] = 2886, 2887, 2894, 2900, 2902, 2904, 2908, 2917, 2922, 2929, 2944, 2951, 2955, 2959, 2963, 2967, 2977, 2986, 3008, 3009, 3013, 3014, 3015, 3019, 3020, 3027, 3028, 3032, 3033, 3038, - 3046, 3048, 3062, 3065, 3092, 3093, 3096, 3097, 3108, 3114, - 3121, 3130, 3147, 3192, 3200, 3208, 3216, 3224, 3245, 3246, - 3249, 3250, 3254, 3264, 3265, 3269, 3270, 3274, 3281, 3282, - 3287, 3288, 3293, 3294, 3297, 3298, 3299, 3302, 3303, 3306, - 3307, 3308, 3309, 3310, 3311, 3312, 3313, 3314, 3315, 3316, - 3317, 3318, 3319, 3322, 3324, 3329, 3331, 3336, 3338, 3340, - 3342, 3344, 3346, 3348, 3350, 3364, 3366, 3371, 3375, 3382, - 3387, 3393, 3397, 3404, 3409, 3416, 3421, 3429, 3433, 3439, - 3443, 3452, 3463, 3464, 3468, 3472, 3479, 3480, 3481, 3482, - 3483, 3484, 3485, 3486, 3487, 3488, 3498, 3502, 3509, 3516, - 3517, 3533, 3537, 3542, 3546, 3561, 3566, 3570, 3573, 3576, - 3577, 3578, 3581, 3588, 3598, 3612, 3613, 3617, 3628, 3629, - 3632, 3633, 3636, 3640, 3647, 3655, 3663, 3671, 3681, 3682, - 3687, 3688, 3692, 3693, 3694, 3698, 3707, 3715, 3723, 3732, - 3747, 3748, 3753, 3754, 3764, 3765, 3769, 3770, 3774, 3775, - 3778, 3794, 3802, 3812, 3813, 3816, 3817, 3820, 3824, 3825, - 3829, 3830, 3833, 3834, 3835, 3845, 3846, 3850, 3852, 3858, - 3859, 3863, 3864, 3867, 3878, 3881, 3892, 3896, 3900, 3912, - 3916, 3925, 3932, 3970, 3974, 3978, 3982, 3986, 3990, 3994, - 4000, 4017, 4018, 4019, 4022, 4023, 4024, 4027, 4028, 4029, - 4032, 4033, 4036, 4038, 4043, 4044, 4047, 4051, 4052, 7, - 18, 19, 23, 24, 25, 26, 27, 7, 26, 50, - 73, 80, 85, 86, 87, 88, 8, 33, 62, 66, - 67, 72, 73, 78, 79, 83, 84, 89, 90, 7, - 16, 25, 34, 43, 52, 5, 12, 22, 23, 7, - 19, 33, 9, 16, 26, 33, 44, 45, 50, 51, - 52, 57, 58, 59, 60, 61, 62, 63, 64, 65, - 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, - 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, - 86, 90, 91, 92, 97, 98, 103, 107, 115, 116, - 121, 122, 123, 129, 134, 142, 143, 10, 16, 22, - 28, 38, 39, 47, 58, 70, 78, 86, 93, 103, - 105, 111, 115, 119, 134, 141, 142, 143, 147, 148, - 7, 15, 22, 29, 36, 45, 46, 48, 49, 8, - 22, 36, 48, 56, 70, 71, 72, 73, 74, 87, - 88, 93, 94, 98, 99, 7, 18, 31, 35, 42, - 53, 54, 60, 61, 9, 19, 7, 18, 25, 34, - 35, 39, 40, 2, 7, 12, 17, 26, 33, 43, - 44, 51, 3, 10, 17, 24, 31, 38, 45, 52, - 61, 61, 63, 63, 65, 66, 70, 71, 6, 8, - 21, 34, 47, 65, 87, 88, 89, 90, 11, 24, - 37, 54, 55, 56, 61, 74, 74, 74, 74, 74, + 3046, 3048, 3062, 3065, 3092, 3093, 3096, 3097, 3105, 3113, + 3121, 3130, 3140, 3158, 3204, 3213, 3222, 3231, 3240, 3252, + 3253, 3254, 3255, 3256, 3270, 3271, 3274, 3275, 3279, 3289, + 3290, 3294, 3295, 3299, 3306, 3307, 3312, 3313, 3318, 3319, + 3322, 3323, 3324, 3327, 3328, 3331, 3332, 3333, 3334, 3335, + 3336, 3337, 3338, 3339, 3340, 3341, 3342, 3343, 3344, 3347, + 3349, 3354, 3356, 3361, 3363, 3365, 3367, 3369, 3371, 3373, + 3375, 3389, 3391, 3396, 3400, 3407, 3412, 3418, 3422, 3429, + 3434, 3441, 3446, 3454, 3458, 3464, 3468, 3477, 3488, 3489, + 3493, 3497, 3504, 3505, 3506, 3507, 3508, 3509, 3510, 3511, + 3512, 3513, 3523, 3527, 3534, 3541, 3542, 3558, 3562, 3567, + 3571, 3586, 3591, 3595, 3598, 3601, 3602, 3603, 3606, 3613, + 3623, 3637, 3638, 3642, 3653, 3654, 3657, 3658, 3661, 3665, + 3672, 3680, 3688, 3696, 3706, 3707, 3712, 3713, 3717, 3718, + 3719, 3723, 3732, 3740, 3748, 3757, 3772, 3773, 3778, 3779, + 3789, 3790, 3794, 3795, 3799, 3800, 3803, 3819, 3827, 3837, + 3838, 3841, 3842, 3845, 3849, 3850, 3854, 3855, 3858, 3859, + 3860, 3870, 3871, 3875, 3877, 3883, 3884, 3888, 3889, 3892, + 3903, 3906, 3917, 3921, 3925, 3937, 3941, 3950, 3957, 3995, + 3999, 4003, 4007, 4011, 4015, 4019, 4025, 4042, 4043, 4044, + 4047, 4048, 4049, 4052, 4053, 4054, 4057, 4058, 4061, 4063, + 4068, 4069, 4072, 4076, 4077, 7, 18, 19, 23, 24, + 25, 26, 27, 7, 26, 50, 73, 80, 85, 86, + 87, 88, 8, 33, 62, 66, 67, 72, 73, 78, + 79, 83, 84, 89, 90, 7, 16, 25, 34, 43, + 52, 5, 12, 22, 23, 7, 19, 33, 9, 16, + 26, 33, 44, 45, 50, 51, 52, 57, 58, 59, + 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, + 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, + 80, 81, 82, 83, 84, 85, 86, 90, 91, 92, + 97, 98, 103, 107, 115, 116, 121, 122, 123, 129, + 134, 142, 143, 10, 16, 22, 28, 38, 39, 47, + 58, 70, 78, 86, 93, 103, 105, 111, 115, 119, + 134, 141, 142, 143, 147, 148, 7, 15, 22, 29, + 36, 45, 46, 48, 49, 8, 22, 36, 48, 56, + 70, 71, 72, 73, 74, 87, 88, 93, 94, 98, + 99, 7, 18, 31, 35, 42, 53, 54, 60, 61, + 9, 19, 7, 18, 25, 34, 35, 39, 40, 2, + 7, 12, 17, 26, 33, 43, 44, 51, 3, 10, + 17, 24, 31, 38, 45, 52, 61, 61, 63, 63, + 65, 66, 70, 71, 6, 8, 21, 34, 47, 65, + 87, 88, 89, 90, 11, 24, 37, 54, 55, 56, + 61, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, @@ -2637,18 +2648,18 @@ static const yytype_uint16 yyrline[] = 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, - 74, 74, 74, 74, 75, 75, 75, 75, 75, 75, + 74, 74, 74, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, - 75, 75, 75, 75, 75, 75, 75, 75, 76, 76, + 75, 75, 75, 75, 75, 75, 75, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, - 76, 76, 76, 76, 76, 76, 77, 77, 77, 77, + 76, 76, 76, 76, 76, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, - 77, 77, 77, 77, 77, 78, 78, 78, 78, 78, + 77, 77, 77, 77, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, @@ -2656,17 +2667,17 @@ static const yytype_uint16 yyrline[] = 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, - 78, 78, 78, 78, 78, 78, 79, 79, 79, 79, + 78, 78, 78, 78, 78, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, - 79, 79, 79, 79, 79, 79, 79, 79, 80, 80, + 79, 79, 79, 79, 79, 79, 79, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, - 80, 80, 80, 80, 80, 80, 80, 80, 80, 80 + 80, 80, 80, 80, 80, 80, 80, 80, 80 }; #endif @@ -2703,59 +2714,59 @@ static const char *const yytname[] = "FILTER", "FIRST_P", "FLOAT_P", "FOLLOWING", "FOR", "FORCE", "FOREIGN", "FORWARD", "FREEZE", "FROM", "FULL", "FUNCTION", "FUNCTIONS", "GENERATED", "GLOB", "GLOBAL", "GRANT", "GRANTED", "GROUP_P", "GROUPING", - "GROUPING_ID", "HANDLER", "HAVING", "HEADER_P", "HOLD", "HOUR_P", - "HOURS_P", "IDENTITY_P", "IF_P", "IGNORE_P", "ILIKE", "IMMEDIATE", - "IMMUTABLE", "IMPLICIT_P", "IMPORT_P", "IN_P", "INCLUDE_P", "INCLUDING", - "INCREMENT", "INDEX", "INDEXES", "INHERIT", "INHERITS", "INITIALLY", - "INLINE_P", "INNER_P", "INOUT", "INPUT_P", "INSENSITIVE", "INSERT", - "INSTALL", "INSTEAD", "INT_P", "INTEGER", "INTERSECT", "INTERVAL", - "INTO", "INVOKER", "IS", "ISNULL", "ISOLATION", "JOIN", "JSON", "KEY", - "LABEL", "LANGUAGE", "LARGE_P", "LAST_P", "LATERAL_P", "LEADING", - "LEAKPROOF", "LEFT", "LEVEL", "LIKE", "LIMIT", "LISTEN", "LOAD", "LOCAL", - "LOCATION", "LOCK_P", "LOCKED", "LOGGED", "MACRO", "MAP", "MAPPING", - "MATCH", "MATERIALIZED", "MAXVALUE", "METHOD", "MICROSECOND_P", - "MICROSECONDS_P", "MILLISECOND_P", "MILLISECONDS_P", "MINUTE_P", - "MINUTES_P", "MINVALUE", "MODE", "MONTH_P", "MONTHS_P", "MOVE", "NAME_P", - "NAMES", "NATIONAL", "NATURAL", "NCHAR", "NEW", "NEXT", "NO", "NONE", - "NOT", "NOTHING", "NOTIFY", "NOTNULL", "NOWAIT", "NULL_P", "NULLIF", - "NULLS_P", "NUMERIC", "OBJECT_P", "OF", "OFF", "OFFSET", "OIDS", "OLD", - "ON", "ONLY", "OPERATOR", "OPTION", "OPTIONS", "OR", "ORDER", - "ORDINALITY", "OUT_P", "OUTER_P", "OVER", "OVERLAPS", "OVERLAY", - "OVERRIDING", "OWNED", "OWNER", "PARALLEL", "PARSER", "PARTIAL", - "PARTITION", "PASSING", "PASSWORD", "PERCENT", "PIVOT", "PIVOT_LONGER", - "PIVOT_WIDER", "PLACING", "PLANS", "POLICY", "POSITION", "POSITIONAL", - "PRAGMA_P", "PRECEDING", "PRECISION", "PREPARE", "PREPARED", "PRESERVE", - "PRIMARY", "PRIOR", "PRIVILEGES", "PROCEDURAL", "PROCEDURE", "PROGRAM", - "PUBLICATION", "QUALIFY", "QUOTE", "RANGE", "READ_P", "REAL", "REASSIGN", - "RECHECK", "RECURSIVE", "REF", "REFERENCES", "REFERENCING", "REFRESH", - "REINDEX", "RELATIVE_P", "RELEASE", "RENAME", "REPEATABLE", "REPLACE", - "REPLICA", "RESET", "RESPECT_P", "RESTART", "RESTRICT", "RETURNING", - "RETURNS", "REVOKE", "RIGHT", "ROLE", "ROLLBACK", "ROLLUP", "ROW", - "ROWS", "RULE", "SAMPLE", "SAVEPOINT", "SCHEMA", "SCHEMAS", "SCROLL", - "SEARCH", "SECOND_P", "SECONDS_P", "SECURITY", "SELECT", "SEMI", - "SEQUENCE", "SEQUENCES", "SERIALIZABLE", "SERVER", "SESSION", "SET", - "SETOF", "SETS", "SHARE", "SHOW", "SIMILAR", "SIMPLE", "SKIP", - "SMALLINT", "SNAPSHOT", "SOME", "SQL_P", "STABLE", "STANDALONE_P", - "START", "STATEMENT", "STATISTICS", "STDIN", "STDOUT", "STORAGE", - "STORED", "STRICT_P", "STRIP_P", "STRUCT", "SUBSCRIPTION", "SUBSTRING", - "SUMMARIZE", "SYMMETRIC", "SYSID", "SYSTEM_P", "TABLE", "TABLES", - "TABLESAMPLE", "TABLESPACE", "TEMP", "TEMPLATE", "TEMPORARY", "TEXT_P", - "THEN", "TIME", "TIMESTAMP", "TO", "TRAILING", "TRANSACTION", - "TRANSFORM", "TREAT", "TRIGGER", "TRIM", "TRUE_P", "TRUNCATE", "TRUSTED", - "TRY_CAST", "TYPE_P", "TYPES_P", "UNBOUNDED", "UNCOMMITTED", - "UNENCRYPTED", "UNION", "UNIQUE", "UNKNOWN", "UNLISTEN", "UNLOGGED", - "UNPIVOT", "UNTIL", "UPDATE", "USE_P", "USER", "USING", "VACUUM", - "VALID", "VALIDATE", "VALIDATOR", "VALUE_P", "VALUES", "VARCHAR", - "VARIADIC", "VARYING", "VERBOSE", "VERSION_P", "VIEW", "VIEWS", - "VIRTUAL", "VOLATILE", "WHEN", "WHERE", "WHITESPACE_P", "WINDOW", "WITH", - "WITHIN", "WITHOUT", "WORK", "WRAPPER", "WRITE_P", "XML_P", - "XMLATTRIBUTES", "XMLCONCAT", "XMLELEMENT", "XMLEXISTS", "XMLFOREST", - "XMLNAMESPACES", "XMLPARSE", "XMLPI", "XMLROOT", "XMLSERIALIZE", - "XMLTABLE", "YEAR_P", "YEARS_P", "YES_P", "ZONE", "NOT_LA", "NULLS_LA", - "WITH_LA", "'<'", "'>'", "'='", "POSTFIXOP", "'+'", "'-'", "'*'", "'/'", - "'%'", "'^'", "UMINUS", "'['", "']'", "'('", "')'", "'.'", "';'", "','", - "'#'", "'$'", "'?'", "'{'", "'}'", "':'", "$accept", "stmtblock", - "stmtmulti", "stmt", "AlterTableStmt", + "GROUPING_ID", "GROUPS", "HANDLER", "HAVING", "HEADER_P", "HOLD", + "HOUR_P", "HOURS_P", "IDENTITY_P", "IF_P", "IGNORE_P", "ILIKE", + "IMMEDIATE", "IMMUTABLE", "IMPLICIT_P", "IMPORT_P", "IN_P", "INCLUDE_P", + "INCLUDING", "INCREMENT", "INDEX", "INDEXES", "INHERIT", "INHERITS", + "INITIALLY", "INLINE_P", "INNER_P", "INOUT", "INPUT_P", "INSENSITIVE", + "INSERT", "INSTALL", "INSTEAD", "INT_P", "INTEGER", "INTERSECT", + "INTERVAL", "INTO", "INVOKER", "IS", "ISNULL", "ISOLATION", "JOIN", + "JSON", "KEY", "LABEL", "LANGUAGE", "LARGE_P", "LAST_P", "LATERAL_P", + "LEADING", "LEAKPROOF", "LEFT", "LEVEL", "LIKE", "LIMIT", "LISTEN", + "LOAD", "LOCAL", "LOCATION", "LOCK_P", "LOCKED", "LOGGED", "MACRO", + "MAP", "MAPPING", "MATCH", "MATERIALIZED", "MAXVALUE", "METHOD", + "MICROSECOND_P", "MICROSECONDS_P", "MILLISECOND_P", "MILLISECONDS_P", + "MINUTE_P", "MINUTES_P", "MINVALUE", "MODE", "MONTH_P", "MONTHS_P", + "MOVE", "NAME_P", "NAMES", "NATIONAL", "NATURAL", "NCHAR", "NEW", "NEXT", + "NO", "NONE", "NOT", "NOTHING", "NOTIFY", "NOTNULL", "NOWAIT", "NULL_P", + "NULLIF", "NULLS_P", "NUMERIC", "OBJECT_P", "OF", "OFF", "OFFSET", + "OIDS", "OLD", "ON", "ONLY", "OPERATOR", "OPTION", "OPTIONS", "OR", + "ORDER", "ORDINALITY", "OTHERS", "OUT_P", "OUTER_P", "OVER", "OVERLAPS", + "OVERLAY", "OVERRIDING", "OWNED", "OWNER", "PARALLEL", "PARSER", + "PARTIAL", "PARTITION", "PASSING", "PASSWORD", "PERCENT", "PIVOT", + "PIVOT_LONGER", "PIVOT_WIDER", "PLACING", "PLANS", "POLICY", "POSITION", + "POSITIONAL", "PRAGMA_P", "PRECEDING", "PRECISION", "PREPARE", + "PREPARED", "PRESERVE", "PRIMARY", "PRIOR", "PRIVILEGES", "PROCEDURAL", + "PROCEDURE", "PROGRAM", "PUBLICATION", "QUALIFY", "QUOTE", "RANGE", + "READ_P", "REAL", "REASSIGN", "RECHECK", "RECURSIVE", "REF", + "REFERENCES", "REFERENCING", "REFRESH", "REINDEX", "RELATIVE_P", + "RELEASE", "RENAME", "REPEATABLE", "REPLACE", "REPLICA", "RESET", + "RESPECT_P", "RESTART", "RESTRICT", "RETURNING", "RETURNS", "REVOKE", + "RIGHT", "ROLE", "ROLLBACK", "ROLLUP", "ROW", "ROWS", "RULE", "SAMPLE", + "SAVEPOINT", "SCHEMA", "SCHEMAS", "SCROLL", "SEARCH", "SECOND_P", + "SECONDS_P", "SECURITY", "SELECT", "SEMI", "SEQUENCE", "SEQUENCES", + "SERIALIZABLE", "SERVER", "SESSION", "SET", "SETOF", "SETS", "SHARE", + "SHOW", "SIMILAR", "SIMPLE", "SKIP", "SMALLINT", "SNAPSHOT", "SOME", + "SQL_P", "STABLE", "STANDALONE_P", "START", "STATEMENT", "STATISTICS", + "STDIN", "STDOUT", "STORAGE", "STORED", "STRICT_P", "STRIP_P", "STRUCT", + "SUBSCRIPTION", "SUBSTRING", "SUMMARIZE", "SYMMETRIC", "SYSID", + "SYSTEM_P", "TABLE", "TABLES", "TABLESAMPLE", "TABLESPACE", "TEMP", + "TEMPLATE", "TEMPORARY", "TEXT_P", "THEN", "TIES", "TIME", "TIMESTAMP", + "TO", "TRAILING", "TRANSACTION", "TRANSFORM", "TREAT", "TRIGGER", "TRIM", + "TRUE_P", "TRUNCATE", "TRUSTED", "TRY_CAST", "TYPE_P", "TYPES_P", + "UNBOUNDED", "UNCOMMITTED", "UNENCRYPTED", "UNION", "UNIQUE", "UNKNOWN", + "UNLISTEN", "UNLOGGED", "UNPIVOT", "UNTIL", "UPDATE", "USE_P", "USER", + "USING", "VACUUM", "VALID", "VALIDATE", "VALIDATOR", "VALUE_P", "VALUES", + "VARCHAR", "VARIADIC", "VARYING", "VERBOSE", "VERSION_P", "VIEW", + "VIEWS", "VIRTUAL", "VOLATILE", "WHEN", "WHERE", "WHITESPACE_P", + "WINDOW", "WITH", "WITHIN", "WITHOUT", "WORK", "WRAPPER", "WRITE_P", + "XML_P", "XMLATTRIBUTES", "XMLCONCAT", "XMLELEMENT", "XMLEXISTS", + "XMLFOREST", "XMLNAMESPACES", "XMLPARSE", "XMLPI", "XMLROOT", + "XMLSERIALIZE", "XMLTABLE", "YEAR_P", "YEARS_P", "YES_P", "ZONE", + "NOT_LA", "NULLS_LA", "WITH_LA", "'<'", "'>'", "'='", "POSTFIXOP", "'+'", + "'-'", "'*'", "'/'", "'%'", "'^'", "UMINUS", "'['", "']'", "'('", "')'", + "'.'", "';'", "','", "'#'", "'$'", "'?'", "'{'", "'}'", "':'", "$accept", + "stmtblock", "stmtmulti", "stmt", "AlterTableStmt", "alter_identity_column_option_list", "alter_column_default", "alter_identity_column_option", "alter_generic_option_list", "alter_table_cmd", "alter_using", "alter_generic_option_elem", @@ -2837,8 +2848,9 @@ static const char *const yytname[] = "export_clause", "window_clause", "window_definition_list", "window_definition", "over_clause", "window_specification", "opt_existing_window_name", "opt_partition_clause", "opt_frame_clause", - "frame_extent", "frame_bound", "qualified_row", "row", "dict_arg", - "dict_arguments", "dict_arguments_opt_comma", "map_arg", "map_arguments", + "frame_extent", "frame_bound", "opt_window_exclusion_clause", + "qualified_row", "row", "dict_arg", "dict_arguments", + "dict_arguments_opt_comma", "map_arg", "map_arguments", "map_arguments_opt_comma", "opt_map_arguments_opt_comma", "sub_type", "all_Op", "MathOp", "qual_Op", "qual_all_Op", "subquery_Op", "any_operator", "c_expr_list", "c_expr_list_opt_comma", "expr_list", @@ -2932,202 +2944,171 @@ static const yytype_uint16 yytoknum[] = 705, 706, 707, 708, 709, 710, 711, 712, 713, 714, 715, 716, 717, 718, 719, 720, 721, 722, 723, 724, 725, 726, 727, 728, 729, 730, 731, 732, 733, 734, - 735, 736, 737, 738, 739, 740, 741, 60, 62, 61, - 742, 43, 45, 42, 47, 37, 94, 743, 91, 93, - 40, 41, 46, 59, 44, 35, 36, 63, 123, 125, - 58 + 735, 736, 737, 738, 739, 740, 741, 742, 743, 744, + 60, 62, 61, 745, 43, 45, 42, 47, 37, 94, + 746, 91, 93, 40, 41, 46, 59, 44, 35, 36, + 63, 123, 125, 58 }; # endif /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives. */ static const yytype_uint16 yyr1[] = { - 0, 511, 512, 513, 513, 514, 514, 514, 514, 514, - 514, 514, 514, 514, 514, 514, 514, 514, 514, 514, - 514, 514, 514, 514, 514, 514, 514, 514, 514, 514, - 514, 514, 514, 514, 514, 514, 514, 514, 514, 514, - 514, 514, 514, 515, 515, 515, 515, 515, 515, 515, - 515, 516, 516, 517, 517, 518, 518, 518, 518, 519, - 519, 520, 520, 520, 520, 520, 520, 520, 520, 520, - 520, 520, 520, 520, 520, 520, 520, 520, 520, 520, - 520, 520, 520, 520, 520, 520, 520, 520, 520, 520, - 521, 521, 522, 522, 522, 522, 523, 523, 524, 525, - 525, 525, 526, 526, 526, 526, 527, 527, 528, 528, - 528, 529, 529, 530, 531, 531, 532, 533, 534, 534, - 534, 534, 535, 535, 535, 535, 535, 535, 535, 535, - 535, 535, 535, 535, 535, 536, 536, 537, 538, 538, - 538, 538, 538, 539, 539, 540, 540, 540, 541, 541, - 541, 542, 542, 543, 544, 544, 545, 545, 545, 546, - 546, 546, 547, 547, 547, 548, 548, 549, 549, 550, - 550, 551, 551, 552, 552, 553, 553, 554, 554, 555, - 555, 556, 556, 557, 558, 558, 558, 559, 559, 560, - 560, 561, 561, 561, 562, 562, 562, 563, 563, 564, - 564, 564, 565, 565, 566, 566, 567, 567, 568, 568, - 569, 569, 570, 570, 570, 571, 571, 571, 571, 572, - 572, 572, 572, 572, 572, 572, 572, 572, 572, 572, - 572, 572, 572, 573, 573, 574, 574, 574, 575, 575, - 575, 575, 575, 575, 576, 576, 576, 577, 578, 578, - 578, 579, 579, 580, 580, 580, 580, 580, 580, 581, - 581, 582, 583, 583, 583, 583, 583, 584, 584, 584, - 584, 585, 585, 585, 585, 585, 585, 585, 585, 586, - 586, 587, 587, 588, 588, 588, 589, 590, 591, 591, - 591, 591, 591, 592, 592, 592, 592, 593, 594, 594, - 595, 595, 596, 596, 596, 596, 596, 596, 596, 596, - 597, 597, 598, 599, 599, 599, 599, 600, 600, 600, - 600, 601, 602, 602, 602, 603, 604, 604, 604, 604, - 604, 604, 605, 605, 606, 606, 607, 608, 608, 608, - 609, 609, 610, 610, 611, 611, 611, 612, 613, 613, - 614, 614, 615, 616, 616, 616, 616, 617, 617, 618, - 618, 619, 619, 619, 620, 620, 620, 620, 620, 620, - 621, 621, 622, 622, 622, 622, 623, 624, 624, 624, - 624, 624, 624, 624, 624, 625, 625, 626, 626, 626, - 626, 626, 626, 626, 626, 627, 627, 627, 627, 627, - 627, 627, 627, 627, 627, 627, 627, 627, 627, 627, - 627, 627, 628, 628, 628, 628, 628, 628, 629, 629, - 630, 630, 630, 631, 631, 631, 632, 632, 633, 633, - 633, 633, 633, 633, 634, 634, 635, 635, 636, 637, - 637, 638, 638, 639, 639, 640, 640, 641, 641, 642, - 642, 643, 643, 644, 644, 645, 645, 645, 645, 645, - 645, 646, 647, 647, 648, 648, 649, 649, 650, 650, - 650, 650, 650, 650, 650, 650, 650, 650, 650, 650, - 650, 650, 650, 650, 651, 652, 652, 652, 653, 653, - 654, 654, 655, 655, 656, 656, 656, 656, 656, 656, - 656, 656, 657, 657, 658, 658, 659, 659, 659, 659, - 659, 659, 659, 659, 659, 659, 659, 659, 659, 659, - 659, 659, 659, 659, 659, 660, 660, 661, 661, 662, - 662, 663, 663, 663, 664, 664, 665, 665, 666, 666, - 666, 667, 667, 668, 669, 669, 669, 670, 670, 671, - 671, 671, 671, 671, 671, 671, 671, 671, 672, 672, - 673, 673, 673, 674, 675, 675, 676, 676, 677, 677, - 677, 678, 678, 679, 679, 680, 680, 681, 681, 682, - 682, 682, 683, 683, 683, 684, 684, 684, 684, 685, - 685, 686, 686, 686, 686, 687, 687, 688, 688, 688, - 688, 688, 688, 689, 689, 690, 690, 691, 691, 691, - 691, 692, 693, 693, 694, 694, 695, 695, 695, 695, - 695, 696, 697, 697, 697, 698, 698, 699, 699, 700, - 700, 701, 701, 701, 702, 702, 703, 703, 704, 704, - 704, 704, 704, 705, 706, 707, 708, 709, 709, 710, - 710, 711, 711, 712, 712, 713, 713, 714, 714, 715, - 716, 716, 716, 716, 717, 717, 718, 718, 718, 719, - 719, 720, 720, 721, 721, 722, 722, 723, 723, 724, - 724, 724, 724, 724, 724, 724, 724, 724, 724, 725, - 725, 726, 726, 726, 727, 727, 728, 728, 729, 729, - 730, 730, 731, 731, 732, 733, 733, 734, 734, 734, - 734, 734, 734, 734, 734, 734, 734, 734, 735, 735, - 735, 735, 736, 736, 737, 737, 737, 737, 737, 738, - 738, 738, 738, 738, 738, 739, 739, 740, 740, 741, - 741, 741, 741, 742, 742, 743, 744, 744, 745, 745, - 746, 746, 747, 747, 748, 748, 749, 750, 750, 751, - 751, 752, 752, 753, 753, 754, 754, 754, 754, 754, - 754, 754, 754, 754, 755, 755, 755, 756, 756, 756, - 756, 756, 756, 756, 757, 757, 757, 757, 758, 759, - 759, 760, 760, 760, 760, 760, 760, 760, 760, 760, - 760, 760, 761, 761, 762, 762, 763, 763, 764, 765, - 766, 766, 767, 767, 768, 769, 770, 770, 770, 770, - 770, 770, 771, 771, 772, 772, 772, 772, 773, 774, - 774, 774, 775, 775, 776, 776, 777, 777, 778, 778, - 779, 779, 780, 780, 781, 781, 782, 782, 783, 783, - 783, 783, 783, 783, 783, 783, 783, 783, 783, 783, - 783, 783, 783, 783, 784, 784, 784, 784, 784, 784, - 784, 784, 784, 784, 784, 784, 784, 784, 784, 784, - 784, 784, 784, 784, 784, 784, 784, 784, 784, 784, - 784, 784, 784, 784, 784, 784, 784, 784, 784, 784, - 784, 784, 784, 784, 784, 784, 784, 784, 784, 784, - 784, 784, 784, 784, 784, 784, 784, 784, 784, 784, - 784, 784, 784, 784, 784, 784, 784, 784, 784, 784, - 784, 784, 784, 784, 785, 785, 785, 785, 785, 785, - 785, 785, 785, 785, 785, 785, 785, 785, 785, 785, - 785, 785, 785, 785, 785, 785, 785, 785, 785, 786, - 786, 786, 787, 787, 787, 787, 787, 787, 787, 787, - 787, 787, 787, 787, 787, 788, 788, 788, 788, 788, - 788, 789, 790, 790, 790, 790, 790, 790, 791, 791, - 792, 792, 793, 793, 793, 793, 793, 793, 793, 793, - 793, 793, 793, 793, 793, 793, 794, 794, 795, 795, - 796, 796, 796, 797, 797, 798, 798, 799, 799, 800, - 801, 801, 801, 802, 803, 803, 804, 804, 805, 805, - 805, 806, 806, 807, 807, 807, 807, 807, 808, 808, - 809, 809, 810, 811, 811, 812, 812, 813, 814, 814, - 815, 815, 816, 816, 817, 817, 817, 818, 818, 819, - 819, 819, 819, 819, 819, 819, 819, 819, 819, 819, - 819, 819, 819, 820, 820, 821, 821, 822, 822, 822, - 822, 822, 822, 822, 822, 823, 823, 824, 824, 825, - 825, 826, 826, 827, 827, 828, 828, 829, 829, 830, - 830, 830, 831, 831, 832, 832, 833, 833, 833, 833, - 833, 833, 833, 833, 833, 833, 834, 834, 835, 836, - 836, 837, 837, 837, 837, 837, 837, 838, 839, 840, - 840, 840, 841, 841, 842, 843, 843, 844, 845, 845, - 846, 846, 847, 847, 532, 532, 532, 532, 848, 848, - 849, 849, 850, 850, 850, 851, 851, 851, 851, 851, - 852, 852, 853, 853, 854, 854, 855, 855, 856, 856, - 857, 857, 857, 858, 858, 859, 859, 860, 861, 861, - 862, 862, 863, 863, 863, 864, 864, 865, 865, 866, - 866, 867, 867, 868, 869, 869, 870, 870, 870, 870, - 870, 870, 870, 870, 870, 870, 870, 870, 870, 870, - 871, 872, 872, 872, 873, 873, 873, 874, 874, 874, - 875, 875, 876, 876, 877, 877, 878, 879, 879, 880, - 881, 881, 882, 882, 882, 882, 882, 883, 883, 883, - 884, 884, 885, 885, 885, 885, 886, 886, 887, 888, - 888, 889, 889, 890, 890, 891, 891, 892, 892, 893, - 893, 893, 893, 893, 893, 894, 894, 895, 895, 896, - 896, 897, 898, 898, 898, 898, 899, 899, 900, 900, - 900, 901, 901, 901, 901, 901, 901, 901, 901, 901, - 901, 901, 901, 901, 901, 901, 901, 901, 901, 901, - 901, 901, 901, 901, 901, 901, 901, 901, 901, 901, - 901, 902, 902, 902, 903, 903, 904, 904, 905, 905, - 906, 906, 906, 906, 907, 908, 908, 909, 909, 909, - 909, 910, 910, 910, 910, 911, 911, 911, 911, 912, - 912, 913, 913, 913, 913, 913, 913, 913, 914, 914, - 915, 915, 915, 915, 915, 916, 916, 917, 917, 918, - 918, 918, 918, 918, 919, 919, 919, 919, 919, 920, - 920, 921, 921, 922, 922, 923, 923, 924, 924, 924, - 925, 925, 926, 926, 927, 927, 928, 929, 929, 930, - 930, 931, 931, 932, 932, 932, 932, 933, 933, 934, - 934, 934, 935, 935, 935, 935, 935, 935, 935, 935, - 936, 936, 937, 937, 938, 938, 939, 939, 940, 941, - 941, 941, 941, 941, 942, 942, 942, 942, 943, 943, - 943, 944, 944, 944, 945, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 946, 946, 946, 946, 946, 946, - 946, 946, 946, 946, 947, 947, 947, 947, 947, 947, - 947, 947, 947, 947, 947, 947, 947, 947, 947, 947, - 947, 947, 947, 947, 947, 947, 947, 947, 947, 947, - 947, 947, 947, 947, 947, 947, 947, 947, 947, 947, - 947, 947, 947, 947, 947, 947, 947, 947, 947, 947, - 947, 947, 947, 947, 947, 947, 947, 947, 948, 948, - 948, 948, 948, 948, 948, 948, 948, 948, 948, 948, - 948, 948, 948, 948, 948, 948, 948, 948, 948, 948, - 948, 948, 948, 948, 948, 948, 949, 949, 949, 949, - 949, 949, 949, 949, 949, 949, 949, 949, 949, 949, - 949, 949, 949, 949, 949, 949, 949, 949, 949, 949, - 949, 949, 949, 949, 949, 950, 950, 950, 950, 950, + 0, 514, 515, 516, 516, 517, 517, 517, 517, 517, + 517, 517, 517, 517, 517, 517, 517, 517, 517, 517, + 517, 517, 517, 517, 517, 517, 517, 517, 517, 517, + 517, 517, 517, 517, 517, 517, 517, 517, 517, 517, + 517, 517, 517, 518, 518, 518, 518, 518, 518, 518, + 518, 519, 519, 520, 520, 521, 521, 521, 521, 522, + 522, 523, 523, 523, 523, 523, 523, 523, 523, 523, + 523, 523, 523, 523, 523, 523, 523, 523, 523, 523, + 523, 523, 523, 523, 523, 523, 523, 523, 523, 523, + 524, 524, 525, 525, 525, 525, 526, 526, 527, 528, + 528, 528, 529, 529, 529, 529, 530, 530, 531, 531, + 531, 532, 532, 533, 534, 534, 535, 536, 537, 537, + 537, 537, 538, 538, 538, 538, 538, 538, 538, 538, + 538, 538, 538, 538, 538, 539, 539, 540, 541, 541, + 541, 541, 541, 542, 542, 543, 543, 543, 544, 544, + 544, 545, 545, 546, 547, 547, 548, 548, 548, 549, + 549, 549, 550, 550, 550, 551, 551, 552, 552, 553, + 553, 554, 554, 555, 555, 556, 556, 557, 557, 558, + 558, 559, 559, 560, 561, 561, 561, 562, 562, 563, + 563, 564, 564, 564, 565, 565, 565, 566, 566, 567, + 567, 567, 568, 568, 569, 569, 570, 570, 571, 571, + 572, 572, 573, 573, 573, 574, 574, 574, 574, 575, + 575, 575, 575, 575, 575, 575, 575, 575, 575, 575, + 575, 575, 575, 576, 576, 577, 577, 577, 578, 578, + 578, 578, 578, 578, 579, 579, 579, 580, 581, 581, + 581, 582, 582, 583, 583, 583, 583, 583, 583, 584, + 584, 585, 586, 586, 586, 586, 586, 587, 587, 587, + 587, 588, 588, 588, 588, 588, 588, 588, 588, 589, + 589, 590, 590, 591, 591, 591, 592, 593, 594, 594, + 594, 594, 594, 595, 595, 595, 595, 596, 597, 597, + 598, 598, 599, 599, 599, 599, 599, 599, 599, 599, + 600, 600, 601, 602, 602, 602, 602, 603, 603, 603, + 603, 604, 605, 605, 605, 606, 607, 607, 607, 607, + 607, 607, 608, 608, 609, 609, 610, 611, 611, 611, + 612, 612, 613, 613, 614, 614, 614, 615, 616, 616, + 617, 617, 618, 619, 619, 619, 619, 620, 620, 621, + 621, 622, 622, 622, 623, 623, 623, 623, 623, 623, + 624, 624, 625, 625, 625, 625, 626, 627, 627, 627, + 627, 627, 627, 627, 627, 628, 628, 629, 629, 629, + 629, 629, 629, 629, 629, 630, 630, 630, 630, 630, + 630, 630, 630, 630, 630, 630, 630, 630, 630, 630, + 630, 630, 631, 631, 631, 631, 631, 631, 632, 632, + 633, 633, 633, 634, 634, 634, 635, 635, 636, 636, + 636, 636, 636, 636, 637, 637, 638, 638, 639, 640, + 640, 641, 641, 642, 642, 643, 643, 644, 644, 645, + 645, 646, 646, 647, 647, 648, 648, 648, 648, 648, + 648, 649, 650, 650, 651, 651, 652, 652, 653, 653, + 653, 653, 653, 653, 653, 653, 653, 653, 653, 653, + 653, 653, 653, 653, 654, 655, 655, 655, 656, 656, + 657, 657, 658, 658, 659, 659, 659, 659, 659, 659, + 659, 659, 660, 660, 661, 661, 662, 662, 662, 662, + 662, 662, 662, 662, 662, 662, 662, 662, 662, 662, + 662, 662, 662, 662, 662, 663, 663, 664, 664, 665, + 665, 666, 666, 666, 667, 667, 668, 668, 669, 669, + 669, 670, 670, 671, 672, 672, 672, 673, 673, 674, + 674, 674, 674, 674, 674, 674, 674, 674, 675, 675, + 676, 676, 676, 677, 678, 678, 679, 679, 680, 680, + 680, 681, 681, 682, 682, 683, 683, 684, 684, 685, + 685, 685, 686, 686, 686, 687, 687, 687, 687, 688, + 688, 689, 689, 689, 689, 690, 690, 691, 691, 691, + 691, 691, 691, 692, 692, 693, 693, 694, 694, 694, + 694, 695, 696, 696, 697, 697, 698, 698, 698, 698, + 698, 699, 700, 700, 700, 701, 701, 702, 702, 703, + 703, 704, 704, 704, 705, 705, 706, 706, 707, 707, + 707, 707, 707, 708, 709, 710, 711, 712, 712, 713, + 713, 714, 714, 715, 715, 716, 716, 717, 717, 718, + 719, 719, 719, 719, 720, 720, 721, 721, 721, 722, + 722, 723, 723, 724, 724, 725, 725, 726, 726, 727, + 727, 727, 727, 727, 727, 727, 727, 727, 727, 728, + 728, 729, 729, 729, 730, 730, 731, 731, 732, 732, + 733, 733, 734, 734, 735, 736, 736, 737, 737, 737, + 737, 737, 737, 737, 737, 737, 737, 737, 738, 738, + 738, 738, 739, 739, 740, 740, 740, 740, 740, 741, + 741, 741, 741, 741, 741, 742, 742, 743, 743, 744, + 744, 744, 744, 745, 745, 746, 747, 747, 748, 748, + 749, 749, 750, 750, 751, 751, 752, 753, 753, 754, + 754, 755, 755, 756, 756, 757, 757, 757, 757, 757, + 757, 757, 757, 757, 758, 758, 758, 759, 759, 759, + 759, 759, 759, 759, 760, 760, 760, 760, 761, 762, + 762, 763, 763, 763, 763, 763, 763, 763, 763, 763, + 763, 763, 764, 764, 765, 765, 766, 766, 767, 768, + 769, 769, 770, 770, 771, 772, 773, 773, 773, 773, + 773, 773, 774, 774, 775, 775, 775, 775, 776, 777, + 777, 777, 778, 778, 779, 779, 780, 780, 781, 781, + 782, 782, 783, 783, 784, 784, 785, 785, 786, 786, + 786, 786, 786, 786, 786, 786, 786, 786, 786, 786, + 786, 786, 786, 786, 787, 787, 787, 787, 787, 787, + 787, 787, 787, 787, 787, 787, 787, 787, 787, 787, + 787, 787, 787, 787, 787, 787, 787, 787, 787, 787, + 787, 787, 787, 787, 787, 787, 787, 787, 787, 787, + 787, 787, 787, 787, 787, 787, 787, 787, 787, 787, + 787, 787, 787, 787, 787, 787, 787, 787, 787, 787, + 787, 787, 787, 787, 787, 787, 787, 787, 787, 787, + 787, 787, 787, 787, 788, 788, 788, 788, 788, 788, + 788, 788, 788, 788, 788, 788, 788, 788, 788, 788, + 788, 788, 788, 788, 788, 788, 788, 788, 788, 789, + 789, 789, 790, 790, 790, 790, 790, 790, 790, 790, + 790, 790, 790, 790, 790, 791, 791, 791, 791, 791, + 791, 792, 793, 793, 793, 793, 793, 793, 794, 794, + 795, 795, 796, 796, 796, 796, 796, 796, 796, 796, + 796, 796, 796, 796, 796, 796, 797, 797, 798, 798, + 799, 799, 799, 800, 800, 801, 801, 802, 802, 803, + 804, 804, 804, 805, 806, 806, 807, 807, 808, 808, + 808, 808, 809, 809, 810, 810, 810, 810, 810, 811, + 811, 811, 811, 811, 812, 812, 813, 813, 814, 815, + 815, 816, 816, 817, 818, 818, 819, 819, 820, 820, + 821, 821, 821, 822, 822, 823, 823, 823, 823, 823, + 823, 823, 823, 823, 823, 823, 823, 823, 823, 824, + 824, 825, 825, 826, 826, 826, 826, 826, 826, 826, + 826, 827, 827, 828, 828, 829, 829, 830, 830, 831, + 831, 832, 832, 833, 833, 834, 834, 834, 835, 835, + 836, 836, 837, 837, 837, 837, 837, 837, 837, 837, + 837, 837, 838, 838, 839, 840, 840, 841, 841, 841, + 841, 841, 841, 842, 843, 844, 844, 844, 845, 845, + 846, 847, 847, 848, 849, 849, 850, 850, 851, 851, + 535, 535, 535, 535, 852, 852, 853, 853, 854, 854, + 854, 855, 855, 855, 855, 855, 856, 856, 857, 857, + 858, 858, 859, 859, 860, 860, 861, 861, 861, 862, + 862, 863, 863, 864, 865, 865, 866, 866, 867, 867, + 867, 868, 868, 869, 869, 870, 870, 871, 871, 872, + 873, 873, 874, 874, 874, 874, 874, 874, 874, 874, + 874, 874, 874, 874, 874, 874, 875, 876, 876, 876, + 877, 877, 877, 878, 878, 878, 879, 879, 880, 880, + 881, 881, 882, 883, 883, 884, 885, 885, 886, 886, + 886, 886, 886, 887, 887, 887, 888, 888, 889, 889, + 889, 889, 890, 890, 891, 892, 892, 893, 893, 894, + 894, 895, 895, 896, 896, 897, 897, 897, 897, 897, + 897, 898, 898, 899, 899, 900, 900, 901, 902, 902, + 902, 902, 903, 903, 904, 904, 904, 905, 905, 905, + 905, 905, 905, 905, 905, 905, 905, 905, 905, 905, + 905, 905, 905, 905, 905, 905, 905, 905, 905, 905, + 905, 905, 905, 905, 905, 905, 905, 906, 906, 906, + 907, 907, 908, 908, 909, 909, 910, 910, 910, 910, + 911, 912, 912, 913, 913, 913, 913, 914, 914, 914, + 914, 915, 915, 915, 915, 916, 916, 917, 917, 917, + 917, 917, 917, 917, 918, 918, 919, 919, 919, 919, + 919, 920, 920, 921, 921, 922, 922, 922, 922, 922, + 923, 923, 923, 923, 923, 924, 924, 925, 925, 926, + 926, 927, 927, 928, 928, 928, 929, 929, 930, 930, + 931, 931, 932, 933, 933, 934, 934, 935, 935, 936, + 936, 936, 936, 937, 937, 938, 938, 938, 939, 939, + 939, 939, 939, 939, 939, 939, 940, 940, 941, 941, + 942, 942, 943, 943, 944, 945, 945, 945, 945, 945, + 946, 946, 946, 946, 947, 947, 947, 948, 948, 948, + 949, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, @@ -3135,17 +3116,49 @@ static const yytype_uint16 yyr1[] = 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, - 950, 950, 950, 950, 950, 950, 951, 951, 951, 951, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, + 950, 950, 950, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, - 951, 951, 951, 951, 951, 951, 951, 951, 952, 952, - 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, - 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, - 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, - 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, + 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, + 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, + 951, 951, 951, 951, 951, 951, 951, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, 952, - 952, 952, 952, 952, 952, 952, 952, 952, 952, 952 + 952, 952, 952, 952, 952, 953, 953, 953, 953, 953, + 953, 953, 953, 953, 953, 953, 953, 953, 953, 953, + 953, 953, 953, 953, 953, 953, 953, 953, 953, 953, + 953, 953, 953, 953, 954, 954, 954, 954, 954, 954, + 954, 954, 954, 954, 954, 954, 954, 954, 954, 954, + 954, 954, 954, 954, 954, 954, 954, 954, 954, 954, + 954, 954, 954, 954, 954, 954, 954, 954, 954, 954, + 954, 954, 954, 954, 954, 954, 954, 954, 954, 954, + 954, 954, 954, 954, 954, 954, 954, 954, 954, 954, + 954, 954, 954, 954, 954, 954, 954, 954, 954, 954, + 954, 954, 954, 954, 954, 954, 954, 954, 954, 954, + 954, 954, 954, 954, 954, 955, 955, 955, 955, 955, + 955, 955, 955, 955, 955, 955, 955, 955, 955, 955, + 955, 955, 955, 955, 955, 955, 955, 955, 955, 955, + 955, 955, 955, 955, 955, 955, 955, 956, 956, 956, + 956, 956, 956, 956, 956, 956, 956, 956, 956, 956, + 956, 956, 956, 956, 956, 956, 956, 956, 956, 956, + 956, 956, 956, 956, 956, 956, 956, 956, 956, 956, + 956, 956, 956, 956, 956, 956, 956, 956, 956, 956, + 956, 956, 956, 956, 956, 956, 956, 956, 956, 956, + 956, 956, 956, 956, 956, 956, 956, 956, 956, 956, + 956, 956, 956, 956, 956, 956, 956, 956, 956 }; /* YYR2[YYN] -- Number of symbols composing right hand side of rule YYN. */ @@ -3253,48 +3266,49 @@ static const yytype_uint8 yyr2[] = 1, 1, 5, 6, 6, 4, 4, 4, 4, 6, 5, 5, 5, 4, 6, 4, 7, 9, 5, 0, 5, 4, 0, 1, 0, 2, 0, 1, 3, 3, - 2, 2, 0, 6, 1, 0, 3, 0, 2, 2, - 0, 1, 4, 2, 2, 2, 2, 2, 4, 3, - 1, 5, 3, 1, 3, 1, 2, 3, 1, 3, - 1, 2, 1, 0, 1, 1, 1, 1, 1, 1, + 2, 2, 0, 6, 1, 0, 3, 0, 3, 3, + 3, 0, 1, 4, 2, 2, 2, 2, 2, 3, + 2, 2, 3, 0, 4, 3, 1, 5, 3, 1, + 3, 1, 2, 3, 1, 3, 1, 2, 1, 0, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 4, 1, 4, 1, 4, 1, - 2, 1, 2, 1, 2, 1, 3, 1, 3, 1, - 2, 1, 3, 1, 2, 1, 0, 1, 3, 1, - 3, 3, 1, 3, 3, 0, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 4, 3, 2, 3, - 0, 3, 3, 2, 2, 1, 0, 2, 2, 3, - 2, 1, 1, 3, 5, 1, 2, 4, 2, 0, - 1, 0, 1, 2, 3, 5, 7, 7, 1, 0, - 0, 2, 0, 2, 3, 3, 3, 5, 7, 7, - 0, 2, 1, 0, 1, 0, 1, 3, 1, 2, - 3, 2, 1, 4, 2, 1, 0, 3, 1, 3, - 1, 2, 4, 2, 0, 1, 3, 1, 3, 1, - 2, 1, 3, 1, 1, 2, 1, 1, 2, 1, - 1, 2, 7, 2, 5, 3, 3, 1, 1, 1, + 4, 1, 4, 1, 4, 1, 2, 1, 2, 1, + 2, 1, 3, 1, 3, 1, 2, 1, 3, 1, + 2, 1, 0, 1, 3, 1, 3, 3, 1, 3, + 3, 0, 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 4, 3, 2, 3, 0, 3, 3, 2, + 2, 1, 0, 2, 2, 3, 2, 1, 1, 3, + 5, 1, 2, 4, 2, 0, 1, 0, 1, 2, + 3, 5, 7, 7, 1, 0, 0, 2, 0, 2, + 3, 3, 3, 5, 7, 7, 0, 2, 1, 0, + 1, 0, 1, 3, 1, 2, 3, 2, 1, 4, + 2, 1, 0, 3, 1, 3, 1, 2, 4, 2, + 0, 1, 3, 1, 3, 1, 2, 1, 3, 1, + 1, 2, 1, 1, 2, 1, 1, 2, 7, 2, + 5, 3, 3, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 2, 2, 3, + 3, 0, 1, 1, 1, 5, 3, 0, 1, 1, + 1, 1, 1, 4, 7, 6, 2, 0, 1, 1, + 1, 1, 13, 16, 1, 2, 0, 1, 0, 1, + 0, 2, 0, 1, 0, 6, 8, 6, 8, 6, + 8, 3, 2, 1, 0, 4, 6, 3, 2, 4, + 3, 5, 1, 0, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 2, 2, 3, 3, 0, 1, 1, 1, 5, - 3, 0, 1, 1, 1, 1, 1, 4, 7, 6, - 2, 0, 1, 1, 1, 1, 13, 16, 1, 2, - 0, 1, 0, 1, 0, 2, 0, 1, 0, 6, - 8, 6, 8, 6, 8, 3, 2, 1, 0, 4, - 6, 3, 2, 4, 3, 5, 1, 0, 1, 1, - 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 3, 1, 1, - 1, 1, 1, 1, 2, 1, 1, 2, 3, 3, - 3, 1, 3, 3, 2, 3, 3, 3, 3, 1, - 1, 1, 1, 3, 5, 1, 1, 1, 1, 3, - 2, 2, 3, 4, 5, 1, 1, 1, 1, 4, - 6, 5, 4, 6, 1, 1, 1, 1, 1, 1, - 0, 1, 3, 1, 0, 7, 3, 1, 2, 3, - 2, 0, 2, 0, 2, 4, 5, 3, 5, 1, - 0, 2, 0, 2, 3, 3, 3, 1, 1, 1, - 2, 3, 2, 2, 2, 2, 3, 4, 3, 1, - 1, 1, 1, 0, 1, 3, 1, 3, 2, 9, - 12, 11, 12, 14, 3, 4, 4, 0, 7, 10, - 9, 2, 3, 0, 4, 1, 1, 1, 1, 1, + 1, 1, 1, 3, 1, 1, 1, 1, 1, 1, + 2, 1, 1, 2, 3, 3, 3, 1, 3, 3, + 2, 3, 3, 3, 3, 1, 1, 1, 1, 3, + 5, 1, 1, 1, 1, 3, 2, 2, 3, 4, + 5, 1, 1, 1, 1, 4, 6, 5, 4, 6, + 1, 1, 1, 1, 1, 1, 0, 1, 3, 1, + 0, 7, 3, 1, 2, 3, 2, 0, 2, 0, + 2, 4, 5, 3, 5, 1, 0, 2, 0, 2, + 3, 3, 3, 1, 1, 1, 2, 3, 2, 2, + 2, 2, 3, 4, 3, 1, 1, 1, 1, 0, + 1, 3, 1, 3, 2, 9, 12, 11, 12, 14, + 3, 4, 4, 0, 7, 10, 9, 2, 3, 0, + 4, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, @@ -3354,7 +3368,7 @@ static const yytype_uint8 yyr2[] = 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 + 1, 1, 1, 1, 1, 1, 1, 1, 1 }; /* YYDEFACT[STATE-NAME] -- Default rule to reduce with in state @@ -3362,3925 +3376,3952 @@ static const yytype_uint8 yyr2[] = means the default is an error. */ static const yytype_uint16 yydefact[] = { - 152, 246, 0, 1319, 1318, 1390, 246, 0, 1268, 246, - 467, 384, 0, 1411, 1390, 0, 246, 0, 152, 0, + 152, 246, 0, 1325, 1324, 1396, 246, 0, 1274, 246, + 467, 384, 0, 1417, 1396, 0, 246, 0, 152, 0, 0, 0, 0, 0, 0, 527, 530, 528, 0, 0, - 0, 246, 567, 0, 1410, 246, 0, 0, 559, 529, - 0, 1370, 0, 0, 0, 0, 0, 2, 4, 7, + 0, 246, 567, 0, 1416, 246, 0, 0, 559, 529, + 0, 1376, 0, 0, 0, 0, 0, 2, 4, 7, 19, 32, 28, 0, 18, 30, 16, 23, 6, 34, 36, 17, 22, 14, 35, 12, 33, 503, 490, 572, 502, 0, 0, 151, 671, 510, 31, 15, 27, 5, - 11, 25, 26, 24, 1277, 39, 29, 37, 20, 8, - 9, 21, 38, 40, 1409, 10, 41, 13, 245, 244, - 238, 0, 0, 0, 0, 0, 1389, 0, 239, 108, - 1435, 1436, 1437, 1438, 1439, 1440, 1441, 1442, 1443, 1444, - 1445, 1798, 1446, 1447, 1448, 1449, 1450, 1799, 1451, 1452, - 1453, 1744, 1745, 1800, 1746, 1747, 1454, 1455, 1456, 1457, - 1458, 1459, 1460, 1461, 1748, 1749, 1462, 1463, 1464, 1465, - 1466, 1750, 1801, 1751, 1467, 1468, 1469, 1470, 1471, 1802, - 1472, 1473, 1474, 1475, 1476, 1477, 1478, 1479, 1480, 1803, - 1481, 1482, 1483, 1484, 1485, 1486, 1487, 1488, 1489, 1490, - 1752, 1753, 1491, 1492, 1493, 1494, 1495, 1496, 1497, 1498, - 1499, 1500, 1501, 1502, 1503, 1504, 1505, 1506, 1507, 1508, - 1509, 1510, 1511, 1512, 1513, 1514, 1515, 1516, 1517, 1518, - 1754, 1519, 1520, 1521, 1522, 1523, 1755, 1524, 1525, 1526, - 1756, 1527, 1528, 1529, 1804, 1805, 1530, 1531, 1757, 1807, - 1532, 1533, 1758, 1759, 1534, 1535, 1536, 1537, 1538, 1539, - 1540, 1541, 1808, 1542, 1543, 1544, 1545, 1546, 1547, 1548, - 1549, 1550, 1551, 1552, 1553, 1809, 1760, 1554, 1555, 1556, - 1557, 1558, 1761, 1762, 1763, 1559, 1810, 1811, 1560, 1812, - 1561, 1562, 1563, 1564, 1565, 1566, 1567, 1813, 1568, 1814, - 1569, 1570, 1571, 1572, 1573, 1574, 1575, 1576, 1764, 1577, - 1578, 1579, 1580, 1581, 1582, 1583, 1584, 1585, 1586, 1587, - 1588, 1589, 1590, 1591, 1592, 1593, 1594, 1765, 1816, 1766, - 1595, 1596, 1597, 1767, 1598, 1599, 1817, 1600, 1768, 1601, - 1769, 1602, 1603, 1604, 1605, 1606, 1607, 1608, 1609, 1610, - 1770, 1818, 1611, 1819, 1771, 1612, 1613, 1614, 1615, 1616, - 1617, 1618, 1619, 1620, 1621, 1622, 1623, 1772, 1820, 1624, - 1625, 1773, 1626, 1627, 1628, 1629, 1630, 1631, 1632, 1633, - 1634, 1635, 1636, 1637, 1774, 1638, 1639, 1640, 1641, 1642, - 1643, 1644, 1645, 1646, 1647, 1648, 1649, 1650, 1651, 1652, - 1653, 1654, 1655, 1656, 1821, 1657, 1658, 1659, 1775, 1660, - 1661, 1662, 1663, 1664, 1665, 1666, 1667, 1668, 1669, 1670, - 1671, 1672, 1673, 1674, 1675, 1676, 1776, 1677, 1678, 1679, - 1822, 1680, 1681, 1777, 1682, 1683, 1684, 1685, 1686, 1687, - 1688, 1689, 1690, 1691, 1692, 1693, 1694, 1778, 1695, 1779, - 1696, 1697, 1698, 1699, 1824, 1700, 1701, 1702, 1703, 1704, - 1780, 1781, 1705, 1706, 1782, 1707, 1783, 1708, 1709, 1784, - 1710, 1711, 1712, 1713, 1714, 1715, 1716, 1717, 1718, 1719, - 1720, 1721, 1722, 1723, 1724, 1725, 1726, 1785, 1786, 1727, - 1825, 1728, 1729, 1730, 1731, 1732, 1733, 1734, 1735, 1736, - 1737, 1738, 1739, 1787, 1788, 1789, 1790, 1791, 1792, 1793, - 1794, 1795, 1796, 1797, 1740, 1741, 1742, 1743, 0, 1418, - 0, 1194, 109, 110, 1216, 108, 1757, 1764, 1778, 1267, - 1266, 109, 241, 466, 0, 0, 0, 0, 0, 0, - 378, 377, 0, 1257, 383, 0, 0, 112, 104, 1626, - 111, 1193, 102, 1389, 0, 0, 404, 405, 0, 414, - 0, 397, 402, 398, 0, 423, 416, 424, 406, 396, - 417, 407, 395, 0, 425, 0, 400, 0, 0, 0, - 242, 207, 152, 0, 1283, 1293, 1302, 1298, 1292, 1300, - 1290, 1296, 1282, 1304, 1291, 1295, 1288, 1305, 1286, 1303, - 1301, 1289, 1297, 1281, 1285, 1272, 1277, 1308, 1299, 1306, - 1294, 1307, 1309, 1284, 1310, 1287, 0, 1268, 0, 0, - 1750, 1801, 1755, 0, 1768, 0, 1771, 1772, 1660, 1779, - 1782, 1783, 1784, 1785, 0, 739, 111, 106, 723, 0, - 505, 675, 685, 723, 728, 990, 751, 991, 0, 113, - 1356, 1355, 1351, 1350, 191, 1231, 1398, 1532, 1571, 1675, - 1780, 1705, 1414, 1399, 1393, 1397, 243, 566, 564, 0, - 1165, 1532, 1571, 1664, 1675, 1780, 1327, 1331, 0, 240, - 1416, 1403, 0, 1404, 111, 511, 558, 0, 247, 1369, - 0, 1374, 0, 1640, 538, 541, 1225, 539, 503, 0, - 1, 152, 0, 158, 0, 562, 562, 0, 562, 0, - 495, 0, 0, 503, 498, 502, 672, 1276, 1384, 1413, - 1780, 1705, 1402, 1405, 1540, 0, 0, 1540, 0, 1540, - 0, 1540, 0, 1392, 1149, 0, 1195, 114, 0, 0, - 349, 382, 381, 380, 379, 384, 1540, 1241, 0, 434, - 435, 0, 0, 0, 0, 0, 1252, 105, 103, 0, - 1387, 412, 413, 0, 403, 399, 401, 0, 1217, 1826, - 1827, 1828, 794, 1829, 823, 801, 823, 823, 1830, 1831, - 1832, 1833, 790, 790, 1506, 803, 1834, 1835, 1836, 1540, - 1837, 1838, 791, 792, 828, 1839, 1840, 1841, 1842, 1843, - 0, 0, 1844, 823, 1845, 790, 1846, 1847, 1848, 795, - 1849, 761, 1850, 0, 1851, 793, 762, 1852, 831, 831, - 1853, 0, 818, 1854, 422, 0, 426, 776, 777, 778, - 779, 804, 805, 780, 810, 811, 815, 781, 863, 790, - 1218, 1219, 1540, 1220, 422, 418, 1540, 422, 1187, 1540, - 0, 0, 199, 1274, 1311, 1855, 1856, 1857, 1858, 1859, - 1860, 1861, 1862, 1864, 1863, 1865, 1866, 1867, 1868, 1869, - 1870, 1871, 1872, 1873, 1874, 1875, 1876, 1877, 1878, 1879, - 1880, 1881, 1882, 1883, 1886, 1884, 1885, 1887, 1888, 1889, - 1890, 1891, 1892, 1893, 1894, 1895, 1896, 1897, 1898, 1899, - 1901, 1900, 1902, 1903, 1904, 1905, 1906, 1907, 1908, 1909, - 1910, 1911, 1912, 1913, 1914, 1915, 1916, 1917, 1918, 1919, - 1920, 1921, 1922, 1923, 1924, 1925, 1926, 1927, 1928, 1929, - 1930, 1931, 1932, 1933, 1934, 1935, 1325, 0, 1326, 1316, - 1280, 1312, 1313, 152, 0, 465, 1265, 1352, 0, 0, - 0, 1105, 723, 728, 0, 0, 741, 0, 1120, 0, - 1126, 0, 0, 0, 723, 510, 0, 685, 740, 107, - 0, 721, 722, 613, 613, 567, 0, 548, 733, 0, - 0, 736, 734, 0, 736, 0, 0, 0, 736, 732, - 693, 0, 613, 0, 721, 724, 613, 0, 743, 1271, - 0, 0, 0, 0, 0, 1396, 1394, 1395, 1400, 0, - 0, 0, 1197, 1199, 1200, 1073, 1210, 976, 0, 1745, - 1746, 1747, 1141, 1748, 1749, 1751, 1752, 1753, 930, 1754, - 1208, 1756, 1758, 1759, 1761, 1762, 1763, 1764, 1765, 1766, - 0, 1209, 1769, 1607, 1774, 1775, 1777, 1780, 1781, 1207, - 1786, 0, 0, 0, 1176, 1096, 0, 0, 0, 975, - 0, 1142, 1150, 971, 0, 0, 784, 785, 806, 807, - 786, 812, 813, 787, 0, 1172, 864, 959, 1160, 978, - 1009, 980, 989, 967, 1040, 960, 0, 970, 962, 1168, - 548, 1166, 0, 963, 1196, 548, 1164, 1330, 1328, 1334, - 1329, 0, 0, 0, 0, 0, 107, 1377, 1376, 1368, - 1366, 1367, 1365, 1364, 1371, 0, 1373, 1277, 1091, 1093, - 0, 540, 0, 0, 0, 493, 492, 3, 0, 0, - 0, 0, 560, 561, 0, 0, 0, 0, 0, 0, - 0, 0, 656, 587, 588, 590, 653, 657, 665, 0, - 0, 0, 0, 0, 499, 0, 1225, 1412, 1408, 1406, - 0, 0, 0, 136, 136, 0, 0, 0, 0, 0, - 96, 45, 89, 0, 0, 0, 0, 221, 234, 0, - 0, 0, 0, 0, 231, 0, 0, 214, 47, 208, - 210, 0, 136, 0, 43, 0, 0, 0, 49, 0, - 465, 1148, 0, 118, 1968, 1969, 1970, 1971, 1972, 1973, - 1974, 1975, 1976, 1977, 1978, 1979, 1980, 1981, 1982, 1983, - 1984, 1985, 1986, 1987, 1988, 1989, 1990, 1991, 1992, 1993, - 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002, 2003, - 2004, 2005, 2006, 2007, 2008, 2009, 2010, 2011, 2012, 2013, - 2014, 2015, 2016, 2017, 2018, 2019, 2020, 2021, 2022, 2023, - 2024, 2025, 2026, 2027, 2028, 2029, 2030, 2031, 2032, 2033, - 2034, 1922, 2035, 2036, 2037, 2038, 2039, 116, 117, 120, - 119, 121, 115, 108, 0, 1936, 1798, 1799, 1800, 1801, - 1751, 1802, 1803, 0, 1804, 1805, 1757, 1807, 1808, 1809, - 1810, 1811, 1812, 1813, 1814, 1764, 1816, 1817, 1818, 1819, - 1820, 1821, 1962, 1822, 1778, 1824, 1784, 0, 1825, 982, - 1099, 572, 1097, 1226, 0, 109, 1213, 0, 0, 463, - 0, 0, 0, 1237, 0, 0, 1540, 198, 1540, 349, - 0, 1540, 349, 1540, 0, 1251, 1254, 0, 415, 411, - 409, 408, 410, 822, 809, 817, 816, 1096, 799, 798, - 797, 0, 796, 0, 0, 823, 823, 821, 800, 776, - 0, 0, 0, 827, 0, 825, 0, 420, 421, 0, - 393, 0, 769, 765, 0, 836, 837, 838, 839, 846, - 847, 844, 845, 840, 841, 834, 835, 842, 843, 832, - 833, 0, 848, 849, 850, 851, 852, 853, 854, 855, - 782, 788, 0, 0, 1221, 0, 388, 0, 0, 390, - 0, 0, 204, 0, 202, 0, 152, 0, 215, 1321, - 1322, 1320, 0, 0, 1315, 1279, 218, 235, 1324, 1314, - 1323, 1278, 1273, 0, 0, 1269, 453, 0, 0, 0, - 0, 1106, 1115, 1107, 1108, 1109, 1110, 1111, 1112, 1113, - 1114, 0, 0, 684, 682, 0, 0, 0, 0, 0, - 0, 1142, 0, 934, 960, 0, 0, 0, 1091, 1125, - 0, 0, 0, 0, 0, 0, 1091, 1131, 0, 0, - 707, 719, 0, 606, 612, 683, 681, 0, 1165, 676, - 0, 753, 0, 733, 0, 732, 0, 0, 735, 729, - 0, 730, 0, 0, 0, 0, 731, 0, 0, 0, - 0, 0, 679, 0, 719, 0, 680, 750, 1358, 1357, - 1353, 1340, 1339, 1348, 192, 0, 1102, 0, 152, 1401, - 1415, 0, 1096, 968, 1140, 0, 0, 973, 1053, 889, - 0, 0, 890, 868, 869, 0, 1175, 1184, 1091, 1095, - 0, 971, 1091, 0, 964, 965, 0, 1043, 1045, 0, - 0, 1143, 1198, 972, 0, 1203, 0, 863, 863, 1171, - 1073, 0, 1063, 1066, 0, 0, 1070, 1071, 1072, 0, - 0, 0, 1163, 0, 1081, 1083, 0, 0, 905, 1079, - 0, 908, 0, 0, 0, 0, 1067, 1068, 1069, 1059, - 1060, 1061, 1062, 1064, 1065, 1077, 1058, 886, 0, 961, - 0, 1012, 0, 885, 1169, 674, 0, 1201, 674, 1342, - 1346, 1347, 1341, 1345, 0, 1333, 1332, 1337, 1335, 1338, - 1336, 1417, 0, 1378, 1362, 0, 1359, 1094, 669, 542, - 1189, 0, 546, 1383, 157, 156, 0, 0, 515, 514, - 581, 573, 575, 581, 0, 513, 0, 629, 630, 0, - 0, 0, 0, 662, 660, 1197, 1210, 617, 591, 616, - 0, 0, 595, 0, 621, 864, 655, 497, 585, 586, - 589, 496, 0, 658, 0, 668, 0, 534, 536, 519, - 533, 531, 516, 524, 656, 590, 0, 1385, 1407, 0, - 0, 0, 0, 0, 1540, 0, 0, 764, 80, 61, - 301, 135, 0, 0, 0, 0, 0, 0, 0, 88, - 85, 86, 87, 0, 0, 0, 0, 219, 220, 233, - 0, 224, 225, 222, 226, 227, 0, 0, 212, 213, - 0, 0, 0, 0, 211, 0, 0, 0, 0, 0, - 0, 0, 0, 1391, 1386, 1144, 1149, 572, 572, 572, - 0, 570, 571, 0, 0, 452, 347, 357, 0, 0, - 0, 1241, 0, 0, 0, 0, 0, 0, 384, 1244, - 1242, 1240, 1243, 1245, 1512, 186, 0, 0, 194, 197, - 0, 346, 320, 0, 0, 1256, 0, 0, 0, 1540, - 336, 1253, 0, 1388, 0, 0, 0, 422, 0, 820, - 819, 770, 766, 0, 0, 0, 0, 0, 0, 427, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 422, - 1222, 0, 419, 422, 1188, 0, 422, 206, 0, 0, - 1275, 1317, 216, 236, 217, 237, 465, 460, 488, 0, - 468, 473, 450, 0, 450, 0, 470, 474, 450, 469, - 0, 450, 464, 1354, 0, 1005, 0, 995, 0, 0, - 742, 0, 0, 996, 936, 937, 0, 0, 0, 0, + 11, 25, 26, 24, 1283, 39, 29, 37, 20, 8, + 9, 21, 38, 40, 1415, 10, 41, 13, 245, 244, + 238, 0, 0, 0, 0, 0, 1395, 0, 239, 108, + 1441, 1442, 1443, 1444, 1445, 1446, 1447, 1448, 1449, 1450, + 1451, 1807, 1452, 1453, 1454, 1455, 1456, 1808, 1457, 1458, + 1459, 1753, 1754, 1809, 1755, 1756, 1460, 1461, 1462, 1463, + 1464, 1465, 1466, 1467, 1757, 1758, 1468, 1469, 1470, 1471, + 1472, 1759, 1810, 1760, 1473, 1474, 1475, 1476, 1477, 1811, + 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1485, 1486, 1812, + 1487, 1488, 1489, 1490, 1491, 1492, 1493, 1494, 1495, 1496, + 1761, 1762, 1497, 1498, 1499, 1500, 1501, 1502, 1503, 1504, + 1505, 1506, 1507, 1508, 1509, 1510, 1511, 1512, 1513, 1514, + 1515, 1516, 1517, 1518, 1519, 1520, 1521, 1522, 1523, 1524, + 1763, 1525, 1526, 1527, 1528, 1529, 1764, 1530, 1531, 1532, + 1765, 1533, 1534, 1535, 1813, 1814, 1536, 1537, 1766, 1816, + 1538, 1539, 1767, 1768, 1540, 1541, 1542, 1543, 1544, 1545, + 1546, 1547, 1548, 1817, 1549, 1550, 1551, 1552, 1553, 1554, + 1555, 1556, 1557, 1558, 1559, 1560, 1818, 1769, 1561, 1562, + 1563, 1564, 1565, 1770, 1771, 1772, 1566, 1819, 1820, 1567, + 1821, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 1822, 1575, + 1823, 1576, 1577, 1578, 1579, 1580, 1581, 1582, 1583, 1773, + 1584, 1585, 1586, 1587, 1588, 1589, 1590, 1591, 1592, 1593, + 1594, 1595, 1596, 1597, 1598, 1599, 1600, 1601, 1774, 1825, + 1775, 1602, 1603, 1604, 1776, 1605, 1606, 1826, 1607, 1777, + 1608, 1778, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 1616, + 1617, 1618, 1779, 1827, 1619, 1828, 1780, 1620, 1621, 1622, + 1623, 1624, 1625, 1626, 1627, 1628, 1629, 1630, 1631, 1781, + 1829, 1632, 1633, 1782, 1634, 1635, 1636, 1637, 1638, 1639, + 1640, 1641, 1642, 1643, 1644, 1645, 1783, 1646, 1647, 1648, + 1649, 1650, 1651, 1652, 1653, 1654, 1655, 1656, 1657, 1658, + 1659, 1660, 1661, 1662, 1663, 1664, 1830, 1665, 1666, 1667, + 1784, 1668, 1669, 1670, 1671, 1672, 1673, 1674, 1675, 1676, + 1677, 1678, 1679, 1680, 1681, 1682, 1683, 1684, 1785, 1685, + 1686, 1687, 1831, 1688, 1689, 1786, 1690, 1691, 1692, 1693, + 1694, 1695, 1696, 1697, 1698, 1699, 1700, 1701, 1702, 1787, + 1703, 1788, 1704, 1705, 1706, 1707, 1833, 1708, 1709, 1710, + 1711, 1712, 1713, 1789, 1790, 1714, 1715, 1791, 1716, 1792, + 1717, 1718, 1793, 1719, 1720, 1721, 1722, 1723, 1724, 1725, + 1726, 1727, 1728, 1729, 1730, 1731, 1732, 1733, 1734, 1735, + 1794, 1795, 1736, 1834, 1737, 1738, 1739, 1740, 1741, 1742, + 1743, 1744, 1745, 1746, 1747, 1748, 1796, 1797, 1798, 1799, + 1800, 1801, 1802, 1803, 1804, 1805, 1806, 1749, 1750, 1751, + 1752, 0, 1424, 0, 1200, 109, 110, 1222, 108, 1766, + 1773, 1787, 1273, 1272, 109, 241, 466, 0, 0, 0, + 0, 0, 0, 378, 377, 0, 1263, 383, 0, 0, + 112, 104, 1634, 111, 1199, 102, 1395, 0, 0, 404, + 405, 0, 414, 0, 397, 402, 398, 0, 423, 416, + 424, 406, 396, 417, 407, 395, 0, 425, 0, 400, + 0, 0, 0, 242, 207, 152, 0, 1289, 1299, 1308, + 1304, 1298, 1306, 1296, 1302, 1288, 1310, 1297, 1301, 1294, + 1311, 1292, 1309, 1307, 1295, 1303, 1287, 1291, 1278, 1283, + 1314, 1305, 1312, 1300, 1313, 1315, 1290, 1316, 1293, 0, + 1274, 0, 0, 1759, 1810, 1764, 0, 1777, 0, 1780, + 1781, 1668, 1788, 1791, 1792, 1793, 1794, 0, 739, 111, + 106, 723, 0, 505, 675, 685, 723, 728, 990, 751, + 991, 0, 113, 1362, 1361, 1357, 1356, 191, 1237, 1404, + 1538, 1578, 1683, 1789, 1714, 1420, 1405, 1399, 1403, 243, + 566, 564, 0, 1171, 1538, 1578, 1672, 1683, 1789, 1333, + 1337, 0, 240, 1422, 1409, 0, 1410, 111, 511, 558, + 0, 247, 1375, 0, 1380, 0, 1648, 538, 541, 1231, + 539, 503, 0, 1, 152, 0, 158, 0, 562, 562, + 0, 562, 0, 495, 0, 0, 503, 498, 502, 672, + 1282, 1390, 1419, 1789, 1714, 1408, 1411, 1547, 0, 0, + 1547, 0, 1547, 0, 1547, 0, 1398, 1155, 0, 1201, + 114, 0, 0, 349, 382, 381, 380, 379, 384, 1547, + 1247, 0, 434, 435, 0, 0, 0, 0, 0, 1258, + 105, 103, 0, 1393, 412, 413, 0, 403, 399, 401, + 0, 1223, 1835, 1836, 1837, 794, 1838, 823, 801, 823, + 823, 1839, 1840, 1841, 1842, 790, 790, 1512, 803, 1843, + 1844, 1845, 1547, 1846, 1847, 791, 792, 828, 1848, 1849, + 1850, 1851, 1852, 0, 0, 1853, 823, 1854, 790, 1855, + 1856, 1857, 795, 1858, 761, 1859, 0, 1860, 793, 762, + 1861, 831, 831, 1862, 0, 818, 1863, 422, 0, 426, + 776, 777, 778, 779, 804, 805, 780, 810, 811, 815, + 781, 863, 790, 1224, 1225, 1547, 1226, 422, 418, 1547, + 422, 1193, 1547, 0, 0, 199, 1280, 1317, 1864, 1865, + 1866, 1867, 1868, 1869, 1870, 1871, 1873, 1872, 1874, 1875, + 1876, 1877, 1878, 1879, 1880, 1881, 1882, 1883, 1884, 1885, + 1886, 1887, 1888, 1889, 1890, 1891, 1892, 1895, 1893, 1894, + 1896, 1897, 1898, 1899, 1900, 1901, 1902, 1903, 1904, 1905, + 1906, 1907, 1908, 1910, 1909, 1911, 1912, 1913, 1914, 1915, + 1916, 1917, 1918, 1919, 1920, 1921, 1922, 1923, 1924, 1925, + 1926, 1927, 1928, 1929, 1930, 1931, 1932, 1933, 1934, 1935, + 1936, 1937, 1938, 1939, 1940, 1941, 1942, 1943, 1944, 1331, + 0, 1332, 1322, 1286, 1318, 1319, 152, 0, 465, 1271, + 1358, 0, 0, 0, 1111, 723, 728, 0, 0, 741, + 0, 1126, 0, 1132, 0, 0, 0, 723, 510, 0, + 685, 740, 107, 0, 721, 722, 613, 613, 567, 0, + 548, 733, 0, 0, 736, 734, 0, 736, 0, 0, + 0, 736, 732, 693, 0, 613, 0, 721, 724, 613, + 0, 743, 1277, 0, 0, 0, 0, 0, 1402, 1400, + 1401, 1406, 0, 0, 0, 1203, 1205, 1206, 1079, 1216, + 976, 0, 1754, 1755, 1756, 1147, 1757, 1758, 1760, 1761, + 1762, 930, 1763, 1214, 1765, 1767, 1768, 1770, 1771, 1772, + 1773, 1774, 1775, 0, 1215, 1778, 1614, 1783, 1784, 1786, + 1789, 1790, 1213, 1795, 0, 0, 0, 1182, 1102, 0, + 0, 0, 975, 0, 1148, 1156, 971, 0, 0, 784, + 785, 806, 807, 786, 812, 813, 787, 0, 1178, 864, + 959, 1166, 978, 1009, 980, 989, 967, 1046, 960, 0, + 970, 962, 1174, 548, 1172, 0, 963, 1202, 548, 1170, + 1336, 1334, 1340, 1335, 0, 0, 0, 0, 0, 107, + 1383, 1382, 1374, 1372, 1373, 1371, 1370, 1377, 0, 1379, + 1283, 1097, 1099, 0, 540, 0, 0, 0, 493, 492, + 3, 0, 0, 0, 0, 560, 561, 0, 0, 0, + 0, 0, 0, 0, 0, 656, 587, 588, 590, 653, + 657, 665, 0, 0, 0, 0, 0, 499, 0, 1231, + 1418, 1414, 1412, 0, 0, 0, 136, 136, 0, 0, + 0, 0, 0, 96, 45, 89, 0, 0, 0, 0, + 221, 234, 0, 0, 0, 0, 0, 231, 0, 0, + 214, 47, 208, 210, 0, 136, 0, 43, 0, 0, + 0, 49, 0, 465, 1154, 0, 118, 1977, 1978, 1979, + 1980, 1981, 1982, 1983, 1984, 1985, 1986, 1987, 1988, 1989, + 1990, 1991, 1992, 1993, 1994, 1995, 1996, 1997, 1998, 1999, + 2000, 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, + 2010, 2011, 2012, 2013, 2014, 2015, 2016, 2017, 2018, 2019, + 2020, 2021, 2022, 2023, 2024, 2025, 2026, 2027, 2028, 2029, + 2030, 2031, 2032, 2033, 2034, 2035, 2036, 2037, 2038, 2039, + 2040, 2041, 2042, 2043, 1931, 2044, 2045, 2046, 2047, 2048, + 116, 117, 120, 119, 121, 115, 108, 0, 1945, 1807, + 1808, 1809, 1810, 1760, 1811, 1812, 0, 1813, 1814, 1766, + 1816, 1817, 1818, 1819, 1820, 1821, 1822, 1823, 1773, 1825, + 1826, 1827, 1828, 1829, 1830, 1971, 1831, 1787, 1833, 1793, + 0, 1834, 982, 1105, 572, 1103, 1232, 0, 109, 1219, + 0, 0, 463, 0, 0, 0, 1243, 0, 0, 1547, + 198, 1547, 349, 0, 1547, 349, 1547, 0, 1257, 1260, + 0, 415, 411, 409, 408, 410, 822, 809, 817, 816, + 1102, 799, 798, 797, 0, 796, 0, 0, 823, 823, + 821, 800, 776, 0, 0, 0, 827, 0, 825, 0, + 420, 421, 0, 393, 0, 769, 765, 0, 836, 837, + 838, 839, 846, 847, 844, 845, 840, 841, 834, 835, + 842, 843, 832, 833, 0, 848, 849, 850, 851, 852, + 853, 854, 855, 782, 788, 0, 0, 1227, 0, 388, + 0, 0, 390, 0, 0, 204, 0, 202, 0, 152, + 0, 215, 1327, 1328, 1326, 0, 0, 1321, 1285, 218, + 235, 1330, 1320, 1329, 1284, 1279, 0, 0, 1275, 453, + 0, 0, 0, 0, 1112, 1121, 1113, 1114, 1115, 1116, + 1117, 1118, 1119, 1120, 0, 0, 684, 682, 0, 0, + 0, 0, 0, 0, 1148, 0, 934, 960, 0, 0, + 0, 1097, 1131, 0, 0, 0, 0, 0, 0, 1097, + 1137, 0, 0, 707, 719, 0, 606, 612, 683, 681, + 0, 1171, 676, 0, 753, 0, 733, 0, 732, 0, + 0, 735, 729, 0, 730, 0, 0, 0, 0, 731, + 0, 0, 0, 0, 0, 679, 0, 719, 0, 680, + 750, 1364, 1363, 1359, 1346, 1345, 1354, 192, 0, 1108, + 0, 152, 1407, 1421, 0, 1102, 968, 1146, 0, 0, + 973, 1059, 889, 0, 0, 890, 868, 869, 0, 1181, + 1190, 1097, 1101, 0, 971, 1097, 0, 964, 965, 0, + 1049, 1051, 0, 0, 1149, 1204, 972, 0, 1209, 0, + 863, 863, 1177, 1079, 0, 1069, 1072, 0, 0, 1076, + 1077, 1078, 0, 0, 0, 1169, 0, 1087, 1089, 0, + 0, 905, 1085, 0, 908, 0, 0, 0, 0, 1073, + 1074, 1075, 1065, 1066, 1067, 1068, 1070, 1071, 1083, 1064, + 886, 0, 961, 0, 1012, 0, 885, 1175, 674, 0, + 1207, 674, 1348, 1352, 1353, 1347, 1351, 0, 1339, 1338, + 1343, 1341, 1344, 1342, 1423, 0, 1384, 1368, 0, 1365, + 1100, 669, 542, 1195, 0, 546, 1389, 157, 156, 0, + 0, 515, 514, 581, 573, 575, 581, 0, 513, 0, + 629, 630, 0, 0, 0, 0, 662, 660, 1203, 1216, + 617, 591, 616, 0, 0, 595, 0, 621, 864, 655, + 497, 585, 586, 589, 496, 0, 658, 0, 668, 0, + 534, 536, 519, 533, 531, 516, 524, 656, 590, 0, + 1391, 1413, 0, 0, 0, 0, 0, 1547, 0, 0, + 764, 80, 61, 301, 135, 0, 0, 0, 0, 0, + 0, 0, 88, 85, 86, 87, 0, 0, 0, 0, + 219, 220, 233, 0, 224, 225, 222, 226, 227, 0, + 0, 212, 213, 0, 0, 0, 0, 211, 0, 0, + 0, 0, 0, 0, 0, 0, 1397, 1392, 1150, 1155, + 572, 572, 572, 0, 570, 571, 0, 0, 452, 347, + 357, 0, 0, 0, 1247, 0, 0, 0, 0, 0, + 0, 384, 1250, 1248, 1246, 1249, 1251, 1518, 186, 0, + 0, 194, 197, 0, 346, 320, 0, 0, 1262, 0, + 0, 0, 1547, 336, 1259, 0, 1394, 0, 0, 0, + 422, 0, 820, 819, 770, 766, 0, 0, 0, 0, + 0, 0, 427, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 422, 1228, 0, 419, 422, 1194, 0, 422, + 206, 0, 0, 1281, 1323, 216, 236, 217, 237, 465, + 460, 488, 0, 468, 473, 450, 0, 450, 0, 470, + 474, 450, 469, 0, 450, 464, 1360, 0, 1005, 0, + 995, 0, 0, 742, 0, 0, 996, 936, 937, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 954, 953, 997, 746, 0, 749, - 0, 0, 1123, 1124, 0, 998, 0, 0, 1130, 0, - 0, 0, 1003, 0, 686, 0, 0, 0, 601, 605, - 608, 0, 611, 548, 504, 1532, 1571, 0, 559, 559, - 559, 557, 547, 0, 633, 0, 0, 0, 708, 0, - 0, 710, 712, 0, 0, 715, 0, 692, 691, 0, - 0, 0, 0, 754, 0, 1193, 0, 0, 193, 0, - 1230, 0, 1233, 1234, 1235, 1232, 1229, 1236, 0, 0, - 0, 1139, 1135, 0, 0, 1048, 1050, 1052, 0, 1057, - 1063, 1066, 1070, 1071, 1072, 1067, 1068, 1069, 1059, 1060, - 1061, 1062, 1064, 1065, 0, 1085, 0, 1039, 0, 0, - 1174, 0, 932, 0, 966, 977, 0, 0, 1046, 981, - 1176, 1151, 0, 0, 1206, 1205, 865, 874, 877, 909, - 910, 881, 882, 883, 887, 1228, 1227, 1170, 0, 1162, - 0, 0, 866, 891, 896, 0, 1132, 926, 0, 914, - 0, 904, 0, 912, 916, 892, 907, 0, 888, 0, - 1163, 1082, 1084, 0, 1080, 0, 878, 879, 880, 870, - 871, 872, 873, 875, 876, 884, 1056, 1054, 1055, 0, - 1149, 0, 1161, 0, 0, 1014, 0, 0, 911, 1167, - 0, 753, 572, 753, 0, 863, 1379, 1225, 1372, 1225, - 1361, 1092, 1190, 1224, 544, 0, 0, 0, 1381, 143, - 147, 0, 1150, 177, 179, 674, 0, 579, 580, 584, - 0, 0, 584, 563, 512, 1775, 1660, 0, 0, 0, - 0, 622, 663, 0, 654, 619, 620, 0, 618, 1197, - 623, 1196, 624, 627, 628, 596, 1185, 664, 666, 0, - 659, 0, 1191, 518, 537, 0, 0, 0, 0, 0, - 501, 500, 670, 0, 46, 0, 1540, 63, 0, 0, - 0, 0, 0, 0, 251, 0, 351, 251, 101, 1540, - 422, 1540, 422, 1439, 1507, 1676, 0, 59, 325, 92, - 0, 129, 354, 0, 310, 82, 97, 122, 0, 0, - 48, 209, 223, 228, 125, 232, 229, 1261, 230, 136, - 0, 44, 0, 123, 0, 1259, 0, 0, 50, 127, - 1263, 0, 1148, 0, 570, 570, 570, 0, 1098, 0, - 0, 0, 1100, 1101, 451, 0, 348, 0, 462, 441, - 442, 452, 1239, 0, 198, 349, 0, 349, 0, 1241, - 0, 0, 188, 184, 0, 0, 0, 0, 347, 339, - 337, 370, 0, 344, 338, 0, 0, 296, 0, 1433, - 0, 0, 0, 0, 436, 0, 0, 0, 0, 0, - 789, 802, 394, 776, 0, 830, 829, 831, 831, 759, - 776, 0, 776, 0, 774, 0, 814, 783, 856, 857, - 858, 859, 860, 861, 862, 387, 1223, 389, 0, 391, - 205, 203, 1270, 457, 0, 456, 461, 459, 455, 454, - 0, 449, 0, 483, 0, 0, 0, 0, 0, 0, - 0, 0, 992, 1104, 0, 1118, 1117, 935, 942, 945, - 949, 950, 951, 1119, 0, 0, 0, 946, 947, 948, - 938, 939, 940, 941, 943, 944, 952, 751, 0, 0, - 745, 1128, 1127, 1121, 1122, 0, 1000, 1001, 1002, 1129, - 0, 0, 720, 599, 597, 600, 602, 598, 0, 0, - 753, 559, 559, 559, 559, 556, 0, 0, 0, 752, - 0, 650, 716, 714, 0, 738, 0, 711, 0, 717, - 0, 702, 0, 709, 758, 725, 0, 0, 727, 1349, - 1103, 565, 969, 0, 0, 1136, 0, 931, 0, 1051, - 979, 0, 1074, 1038, 0, 0, 0, 1183, 0, 1092, - 1042, 1044, 1184, 974, 863, 0, 0, 0, 0, 0, - 0, 0, 915, 906, 0, 913, 917, 0, 0, 0, - 900, 0, 0, 898, 927, 894, 0, 0, 928, 1148, - 0, 1152, 0, 0, 1013, 1022, 677, 673, 633, 570, - 633, 0, 1343, 1363, 1360, 545, 152, 1382, 0, 166, - 0, 0, 0, 0, 169, 183, 180, 1381, 0, 0, - 574, 576, 0, 1075, 584, 578, 626, 625, 0, 594, - 661, 592, 0, 667, 0, 535, 0, 521, 0, 695, - 0, 0, 0, 0, 0, 300, 0, 0, 0, 251, - 0, 359, 0, 366, 0, 0, 351, 332, 81, 0, - 0, 0, 55, 100, 73, 65, 51, 79, 0, 0, - 84, 0, 77, 94, 95, 93, 98, 0, 261, 286, - 0, 0, 297, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1149, 1145, 1149, 0, 0, 0, - 572, 568, 569, 983, 486, 487, 485, 214, 358, 0, - 0, 196, 346, 0, 1256, 0, 1238, 384, 0, 189, - 0, 187, 0, 198, 349, 0, 324, 320, 345, 318, - 317, 319, 0, 1434, 207, 0, 1428, 349, 1255, 0, - 0, 437, 0, 431, 0, 1250, 808, 772, 0, 826, - 824, 773, 0, 771, 767, 775, 422, 0, 445, 484, - 489, 471, 476, 0, 482, 478, 477, 472, 480, 479, - 475, 993, 1004, 1116, 0, 0, 0, 0, 744, 747, - 0, 999, 994, 718, 0, 0, 633, 0, 0, 0, - 0, 550, 549, 555, 0, 0, 1016, 713, 0, 0, - 0, 700, 690, 696, 0, 0, 0, 0, 756, 755, - 726, 0, 1138, 1134, 1047, 1049, 1086, 1173, 1091, 1178, - 1180, 0, 0, 0, 1041, 933, 1204, 867, 0, 0, - 897, 1133, 918, 0, 0, 0, 893, 1074, 0, 0, - 0, 0, 0, 902, 0, 1156, 1149, 0, 1155, 0, - 0, 0, 0, 988, 678, 650, 0, 650, 0, 0, - 1380, 0, 1375, 144, 145, 146, 0, 0, 0, 161, - 138, 0, 0, 178, 166, 154, 582, 583, 0, 577, - 593, 1186, 1192, 520, 0, 971, 0, 0, 517, 0, - 130, 251, 0, 0, 62, 0, 368, 312, 360, 343, - 327, 0, 0, 0, 252, 0, 385, 0, 0, 333, - 0, 0, 0, 0, 313, 0, 0, 272, 0, 0, - 343, 0, 350, 268, 269, 0, 54, 74, 0, 70, - 0, 99, 0, 0, 0, 0, 0, 57, 69, 0, - 52, 0, 422, 422, 60, 1217, 1826, 1827, 1828, 1829, - 1830, 1831, 1832, 1833, 1834, 1835, 1946, 1836, 1837, 1838, - 1839, 1840, 1841, 1842, 1843, 1955, 1844, 258, 1845, 1607, - 1846, 1847, 1848, 1849, 1850, 0, 1851, 762, 1852, 1853, - 2031, 1854, 1059, 1060, 257, 256, 353, 253, 361, 255, - 0, 1218, 254, 356, 311, 126, 1262, 0, 124, 0, - 1260, 133, 131, 128, 1264, 0, 0, 986, 987, 984, - 570, 465, 444, 0, 0, 1433, 0, 0, 0, 1540, - 0, 185, 0, 1256, 195, 346, 0, 376, 296, 371, - 0, 1433, 1431, 0, 1256, 1427, 0, 428, 0, 0, - 0, 768, 760, 392, 458, 0, 481, 955, 0, 0, - 0, 0, 609, 0, 615, 650, 554, 553, 552, 551, - 632, 1482, 1758, 1659, 0, 636, 631, 634, 639, 641, - 640, 642, 638, 649, 0, 652, 737, 1087, 1089, 0, - 0, 0, 0, 701, 0, 703, 0, 705, 0, 757, - 1137, 1181, 1182, 1177, 0, 864, 924, 922, 919, 0, - 920, 901, 0, 0, 899, 895, 0, 929, 0, 0, - 1153, 0, 1008, 0, 1011, 1025, 1021, 1020, 1016, 983, - 1016, 1344, 543, 165, 142, 168, 167, 0, 1150, 175, - 0, 0, 166, 0, 170, 438, 0, 0, 532, 694, - 525, 526, 0, 364, 64, 0, 343, 0, 251, 329, - 328, 331, 326, 330, 0, 386, 0, 0, 270, 0, - 277, 315, 316, 314, 271, 343, 349, 273, 0, 0, - 0, 66, 56, 53, 58, 67, 0, 0, 68, 71, - 758, 83, 76, 1955, 1964, 0, 0, 0, 0, 0, - 1147, 1146, 0, 440, 447, 214, 0, 0, 320, 1430, - 0, 0, 433, 0, 0, 346, 190, 0, 0, 0, - 1433, 0, 0, 248, 0, 293, 0, 200, 1432, 0, - 0, 1419, 0, 0, 1248, 1249, 0, 446, 956, 0, - 957, 748, 0, 0, 607, 1016, 0, 0, 0, 643, - 637, 0, 1015, 1017, 0, 604, 1090, 697, 0, 699, - 0, 723, 572, 0, 723, 706, 1179, 1006, 0, 921, - 925, 923, 903, 1149, 1157, 1149, 1154, 1010, 1024, 1027, - 652, 1202, 652, 0, 0, 153, 0, 0, 150, 137, - 155, 1076, 522, 523, 0, 251, 0, 342, 365, 282, - 260, 0, 0, 0, 267, 274, 375, 276, 0, 75, - 91, 0, 0, 355, 134, 132, 985, 465, 0, 1256, - 296, 1427, 430, 0, 0, 0, 320, 207, 1429, 309, - 302, 303, 304, 305, 306, 307, 308, 323, 322, 294, - 295, 0, 0, 0, 0, 0, 432, 1250, 0, 172, - 181, 0, 172, 958, 610, 0, 652, 0, 0, 0, - 635, 0, 0, 651, 0, 508, 1088, 0, 689, 687, - 570, 0, 688, 0, 0, 0, 0, 572, 604, 604, - 139, 0, 140, 176, 0, 0, 0, 349, 367, 341, - 0, 334, 280, 279, 281, 285, 0, 283, 0, 299, - 0, 292, 260, 0, 78, 0, 362, 439, 443, 0, - 250, 1421, 346, 0, 1427, 296, 1433, 1427, 0, 1424, - 0, 429, 0, 0, 0, 174, 1256, 0, 174, 0, - 604, 645, 0, 644, 1019, 1018, 606, 698, 0, 0, - 1007, 1159, 1158, 0, 1030, 507, 506, 0, 0, 0, - 0, 375, 0, 321, 0, 0, 282, 0, 275, 372, - 373, 374, 0, 288, 278, 289, 72, 90, 363, 0, - 346, 1422, 249, 201, 1420, 1425, 1426, 0, 172, 171, - 581, 173, 753, 182, 581, 614, 509, 646, 603, 0, - 704, 1026, 0, 0, 0, 0, 149, 753, 160, 0, - 292, 340, 335, 259, 284, 298, 0, 0, 0, 290, - 0, 291, 1427, 0, 174, 584, 1246, 584, 1744, 1483, - 1712, 0, 1028, 1031, 1029, 1023, 141, 148, 0, 251, - 264, 0, 263, 0, 352, 287, 1423, 1256, 581, 162, - 163, 0, 1035, 1034, 1033, 1037, 1036, 753, 369, 262, - 266, 265, 753, 584, 0, 159, 1247, 164, 1032 + 0, 0, 0, 0, 0, 0, 0, 954, 953, 997, + 746, 0, 749, 0, 0, 1129, 1130, 0, 998, 0, + 0, 1136, 0, 0, 0, 1003, 0, 686, 0, 0, + 0, 601, 605, 608, 0, 611, 548, 504, 1538, 1578, + 0, 559, 559, 559, 557, 547, 0, 633, 0, 0, + 0, 708, 0, 0, 710, 712, 0, 0, 715, 0, + 692, 691, 0, 0, 0, 0, 754, 0, 1199, 0, + 0, 193, 0, 1236, 0, 1239, 1240, 1241, 1238, 1235, + 1242, 0, 0, 0, 1145, 1141, 0, 0, 1054, 1056, + 1058, 0, 1063, 1069, 1072, 1076, 1077, 1078, 1073, 1074, + 1075, 1065, 1066, 1067, 1068, 1070, 1071, 0, 1091, 0, + 1045, 0, 0, 1180, 0, 932, 0, 966, 977, 0, + 0, 1052, 981, 1182, 1157, 0, 0, 1212, 1211, 865, + 874, 877, 909, 910, 881, 882, 883, 887, 1234, 1233, + 1176, 0, 1168, 0, 0, 866, 891, 896, 0, 1138, + 926, 0, 914, 0, 904, 0, 912, 916, 892, 907, + 0, 888, 0, 1169, 1088, 1090, 0, 1086, 0, 878, + 879, 880, 870, 871, 872, 873, 875, 876, 884, 1062, + 1060, 1061, 0, 1155, 0, 1167, 0, 0, 1014, 0, + 0, 911, 1173, 0, 753, 572, 753, 0, 863, 1385, + 1231, 1378, 1231, 1367, 1098, 1196, 1230, 544, 0, 0, + 0, 1387, 143, 147, 0, 1156, 177, 179, 674, 0, + 579, 580, 584, 0, 0, 584, 563, 512, 1784, 1668, + 0, 0, 0, 0, 622, 663, 0, 654, 619, 620, + 0, 618, 1203, 623, 1202, 624, 627, 628, 596, 1191, + 664, 666, 0, 659, 0, 1197, 518, 537, 0, 0, + 0, 0, 0, 501, 500, 670, 0, 46, 0, 1547, + 63, 0, 0, 0, 0, 0, 0, 251, 0, 351, + 251, 101, 1547, 422, 1547, 422, 1445, 1513, 1684, 0, + 59, 325, 92, 0, 129, 354, 0, 310, 82, 97, + 122, 0, 0, 48, 209, 223, 228, 125, 232, 229, + 1267, 230, 136, 0, 44, 0, 123, 0, 1265, 0, + 0, 50, 127, 1269, 0, 1154, 0, 570, 570, 570, + 0, 1104, 0, 0, 0, 1106, 1107, 451, 0, 348, + 0, 462, 441, 442, 452, 1245, 0, 198, 349, 0, + 349, 0, 1247, 0, 0, 188, 184, 0, 0, 0, + 0, 347, 339, 337, 370, 0, 344, 338, 0, 0, + 296, 0, 1439, 0, 0, 0, 0, 436, 0, 0, + 0, 0, 0, 789, 802, 394, 776, 0, 830, 829, + 831, 831, 759, 776, 0, 776, 0, 774, 0, 814, + 783, 856, 857, 858, 859, 860, 861, 862, 387, 1229, + 389, 0, 391, 205, 203, 1276, 457, 0, 456, 461, + 459, 455, 454, 0, 449, 0, 483, 0, 0, 0, + 0, 0, 0, 0, 0, 992, 1110, 0, 1124, 1123, + 935, 942, 945, 949, 950, 951, 1125, 0, 0, 0, + 946, 947, 948, 938, 939, 940, 941, 943, 944, 952, + 751, 0, 0, 745, 1134, 1133, 1127, 1128, 0, 1000, + 1001, 1002, 1135, 0, 0, 720, 599, 597, 600, 602, + 598, 0, 0, 753, 559, 559, 559, 559, 556, 0, + 0, 0, 752, 0, 650, 716, 714, 0, 738, 0, + 711, 0, 717, 0, 702, 0, 709, 758, 725, 0, + 0, 727, 1355, 1109, 565, 969, 0, 0, 1142, 0, + 931, 0, 1057, 979, 0, 1080, 1044, 0, 0, 0, + 1189, 0, 1098, 1048, 1050, 1190, 974, 863, 0, 0, + 0, 0, 0, 0, 0, 915, 906, 0, 913, 917, + 0, 0, 0, 900, 0, 0, 898, 927, 894, 0, + 0, 928, 1154, 0, 1158, 0, 0, 1013, 1022, 677, + 673, 633, 570, 633, 0, 1349, 1369, 1366, 545, 152, + 1388, 0, 166, 0, 0, 0, 0, 169, 183, 180, + 1387, 0, 0, 574, 576, 0, 1081, 584, 578, 626, + 625, 0, 594, 661, 592, 0, 667, 0, 535, 0, + 521, 0, 695, 0, 0, 0, 0, 0, 300, 0, + 0, 0, 251, 0, 359, 0, 366, 0, 0, 351, + 332, 81, 0, 0, 0, 55, 100, 73, 65, 51, + 79, 0, 0, 84, 0, 77, 94, 95, 93, 98, + 0, 261, 286, 0, 0, 297, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1155, 1151, 1155, + 0, 0, 0, 572, 568, 569, 983, 486, 487, 485, + 214, 358, 0, 0, 196, 346, 0, 1262, 0, 1244, + 384, 0, 189, 0, 187, 0, 198, 349, 0, 324, + 320, 345, 318, 317, 319, 0, 1440, 207, 0, 1434, + 349, 1261, 0, 0, 437, 0, 431, 0, 1256, 808, + 772, 0, 826, 824, 773, 0, 771, 767, 775, 422, + 0, 445, 484, 489, 471, 476, 0, 482, 478, 477, + 472, 480, 479, 475, 993, 1004, 1122, 0, 0, 0, + 0, 744, 747, 0, 999, 994, 718, 0, 0, 633, + 0, 0, 0, 0, 550, 549, 555, 0, 0, 1016, + 713, 0, 0, 0, 700, 690, 696, 0, 0, 0, + 0, 756, 755, 726, 0, 1144, 1140, 1053, 1055, 1092, + 1179, 1097, 1184, 1186, 0, 0, 0, 1047, 933, 1210, + 867, 0, 0, 897, 1139, 918, 0, 0, 0, 893, + 1080, 0, 0, 0, 0, 0, 902, 0, 1162, 1155, + 0, 1161, 0, 0, 0, 0, 988, 678, 650, 0, + 650, 0, 0, 1386, 0, 1381, 144, 145, 146, 0, + 0, 0, 161, 138, 0, 0, 178, 166, 154, 582, + 583, 0, 577, 593, 1192, 1198, 520, 0, 971, 0, + 0, 517, 0, 130, 251, 0, 0, 62, 0, 368, + 312, 360, 343, 327, 0, 0, 0, 252, 0, 385, + 0, 0, 333, 0, 0, 0, 0, 313, 0, 0, + 272, 0, 0, 343, 0, 350, 268, 269, 0, 54, + 74, 0, 70, 0, 99, 0, 0, 0, 0, 0, + 57, 69, 0, 52, 0, 422, 422, 60, 1223, 1835, + 1836, 1837, 1838, 1839, 1840, 1841, 1842, 1843, 1844, 1955, + 1845, 1846, 1847, 1848, 1849, 1850, 1851, 1852, 1964, 1853, + 258, 1854, 1614, 1855, 1856, 1857, 1858, 1859, 0, 1860, + 762, 1861, 1862, 2040, 1863, 1065, 1066, 257, 256, 353, + 253, 361, 255, 0, 1224, 254, 356, 311, 126, 1268, + 0, 124, 0, 1266, 133, 131, 128, 1270, 0, 0, + 986, 987, 984, 570, 465, 444, 0, 0, 1439, 0, + 0, 0, 1547, 0, 185, 0, 1262, 195, 346, 0, + 376, 296, 371, 0, 1439, 1437, 0, 1262, 1433, 0, + 428, 0, 0, 0, 768, 760, 392, 458, 0, 481, + 955, 0, 0, 0, 0, 609, 0, 615, 650, 554, + 553, 552, 551, 632, 1488, 1767, 1667, 0, 636, 631, + 634, 639, 641, 640, 642, 638, 649, 0, 652, 737, + 1093, 1095, 0, 0, 0, 0, 701, 0, 703, 0, + 705, 0, 757, 1143, 1187, 1188, 1183, 0, 864, 924, + 922, 919, 0, 920, 901, 0, 0, 899, 895, 0, + 929, 0, 0, 1159, 0, 1008, 0, 1011, 1025, 1021, + 1020, 1016, 983, 1016, 1350, 543, 165, 142, 168, 167, + 0, 1156, 175, 0, 0, 166, 0, 170, 438, 0, + 0, 532, 694, 525, 526, 0, 364, 64, 0, 343, + 0, 251, 329, 328, 331, 326, 330, 0, 386, 0, + 0, 270, 0, 277, 315, 316, 314, 271, 343, 349, + 273, 0, 0, 0, 66, 56, 53, 58, 67, 0, + 0, 68, 71, 758, 83, 76, 1964, 1973, 0, 0, + 0, 0, 0, 1153, 1152, 0, 440, 447, 214, 0, + 0, 320, 1436, 0, 0, 433, 0, 0, 346, 190, + 0, 0, 0, 1439, 0, 0, 248, 0, 293, 0, + 200, 1438, 0, 0, 1425, 0, 0, 1254, 1255, 0, + 446, 956, 0, 957, 748, 0, 0, 607, 1016, 0, + 0, 0, 643, 637, 0, 1015, 1017, 0, 604, 1096, + 697, 0, 699, 0, 723, 572, 0, 723, 706, 1185, + 1006, 0, 921, 925, 923, 903, 1155, 1163, 1155, 1160, + 1010, 1024, 1027, 652, 1208, 652, 0, 0, 153, 0, + 0, 150, 137, 155, 1082, 522, 523, 0, 251, 0, + 342, 365, 282, 260, 0, 0, 0, 267, 274, 375, + 276, 0, 75, 91, 0, 0, 355, 134, 132, 985, + 465, 0, 1262, 296, 1433, 430, 0, 0, 0, 320, + 207, 1435, 309, 302, 303, 304, 305, 306, 307, 308, + 323, 322, 294, 295, 0, 0, 0, 0, 0, 432, + 1256, 0, 172, 181, 0, 172, 958, 610, 0, 652, + 0, 0, 0, 635, 0, 0, 651, 0, 508, 1094, + 0, 689, 687, 570, 0, 688, 0, 0, 0, 0, + 572, 604, 604, 139, 0, 140, 176, 0, 0, 0, + 349, 367, 341, 0, 334, 280, 279, 281, 285, 0, + 283, 0, 299, 0, 292, 260, 0, 78, 0, 362, + 439, 443, 0, 250, 1427, 346, 0, 1433, 296, 1439, + 1433, 0, 1430, 0, 429, 0, 0, 0, 174, 1262, + 0, 174, 0, 604, 645, 0, 644, 1019, 1018, 606, + 698, 0, 0, 1007, 1165, 1164, 0, 1031, 507, 506, + 0, 0, 0, 0, 375, 0, 321, 0, 0, 282, + 0, 275, 372, 373, 374, 0, 288, 278, 289, 72, + 90, 363, 0, 346, 1428, 249, 201, 1426, 1431, 1432, + 0, 172, 171, 581, 173, 753, 182, 581, 614, 509, + 646, 603, 0, 704, 1026, 0, 0, 0, 0, 0, + 149, 753, 160, 0, 292, 340, 335, 259, 284, 298, + 0, 0, 0, 290, 0, 291, 1433, 0, 174, 584, + 1252, 584, 1753, 1489, 1721, 0, 1043, 1032, 1043, 1043, + 1023, 141, 148, 0, 251, 264, 0, 263, 0, 352, + 287, 1429, 1262, 581, 162, 163, 0, 1036, 1035, 1034, + 1038, 1037, 0, 1030, 1028, 1029, 753, 369, 262, 266, + 265, 753, 584, 0, 0, 1040, 0, 1041, 159, 1253, + 164, 1033, 1039, 1042 }; /* YYDEFGOTO[NTERM-NUM]. */ static const yytype_int16 yydefgoto[] = { - -1, 46, 47, 48, 564, 2514, 2515, 2516, 2166, 1150, - 3244, 2167, 1151, 1152, 2518, 565, 615, 1041, 617, 1042, - 1553, 717, 1257, 1258, 566, 1703, 567, 2719, 2090, 2462, - 3226, 53, 2979, 2093, 1110, 2982, 3190, 2712, 2977, 2463, - 3265, 3320, 2980, 2094, 2095, 3191, 2096, 568, 2570, 2571, - 569, 570, 1778, 571, 1402, 1403, 832, 572, 1779, 1730, - 1511, 1170, 1720, 1416, 573, 100, 60, 574, 2503, 2836, - 3237, 2529, 3374, 2772, 2773, 3234, 3235, 2506, 2169, 3303, - 3304, 2583, 1712, 3298, 2239, 3177, 2173, 2154, 2774, 2247, - 3137, 2877, 2170, 2754, 2240, 3230, 1791, 2241, 3231, 2998, - 2242, 1757, 1782, 2507, 3305, 2174, 1758, 2502, 2837, 1700, - 2243, 3241, 2244, 525, 2758, 575, 557, 558, 824, 1360, - 559, 804, 576, 735, 1788, 577, 578, 2221, 3045, 2607, - 3046, 2302, 2215, 1425, 2296, 1838, 1760, 1426, 515, 1852, - 2608, 2557, 1839, 579, 1043, 68, 69, 957, 70, 2992, - 71, 72, 1677, 1678, 1679, 662, 674, 675, 2086, 1481, - 1922, 667, 1114, 1646, 649, 650, 2211, 689, 1752, 1641, - 1642, 2099, 2470, 1670, 1671, 1123, 1124, 1910, 3205, 1911, - 1912, 1474, 1475, 3084, 1658, 1662, 1663, 2120, 2110, 1649, - 2371, 2915, 2916, 2917, 2918, 2919, 2920, 2921, 1044, 2646, - 3095, 1666, 1667, 1126, 1127, 1128, 1675, 2130, 74, 75, - 2071, 2446, 2447, 621, 2932, 1500, 1680, 2650, 2651, 2652, - 2936, 2937, 2938, 622, 952, 953, 976, 971, 1489, 1931, - 623, 624, 1887, 1888, 2340, 978, 1924, 1942, 1943, 2658, - 1808, 805, 2155, 1516, 1363, 807, 1045, 808, 1338, 1046, - 1342, 810, 1047, 1048, 1049, 813, 1050, 1051, 1052, 816, - 1334, 1053, 1054, 1353, 1382, 1383, 1384, 1385, 1386, 1387, - 1388, 1389, 1390, 1098, 1681, 1056, 1057, 1058, 1059, 1060, - 1061, 626, 1062, 1063, 1601, 2065, 2445, 2925, 3092, 3093, - 2703, 2967, 3119, 3217, 3334, 3362, 3363, 1064, 1065, 1547, - 1548, 1549, 1965, 1966, 1967, 1968, 2059, 1595, 1596, 1066, - 2839, 1598, 1986, 2928, 2929, 1099, 1467, 1540, 1301, 1302, - 1517, 1441, 1442, 1448, 1862, 1456, 1460, 1892, 1893, 1468, - 2027, 1067, 1961, 1962, 2396, 1525, 1068, 1182, 1552, 2698, - 2062, 1599, 2021, 1075, 1069, 1076, 1071, 1536, 1537, 2407, - 2670, 2671, 1992, 2127, 1630, 2132, 2133, 828, 1072, 1073, - 1074, 1303, 501, 819, 3321, 1394, 1104, 1304, 2017, 580, - 984, 1956, 581, 1313, 1771, 582, 3075, 2890, 1326, 1792, - 2252, 526, 583, 584, 510, 81, 82, 83, 698, 1418, - 585, 1419, 1420, 917, 84, 1512, 919, 920, 587, 656, - 657, 1513, 1615, 1514, 588, 632, 1510, 589, 1094, 671, - 1095, 1097, 590, 1088, 2459, 2088, 89, 90, 91, 107, - 1180, 591, 643, 644, 592, 94, 1138, 645, 663, 593, - 594, 3071, 595, 2586, 1320, 511, 503, 504, 821, 1260, - 1306, 1261 + -1, 46, 47, 48, 567, 2517, 2518, 2519, 2169, 1153, + 3247, 2170, 1154, 1155, 2521, 568, 618, 1044, 620, 1045, + 1556, 720, 1260, 1261, 569, 1706, 570, 2722, 2093, 2465, + 3229, 53, 2982, 2096, 1113, 2985, 3193, 2715, 2980, 2466, + 3268, 3323, 2983, 2097, 2098, 3194, 2099, 571, 2573, 2574, + 572, 573, 1781, 574, 1405, 1406, 835, 575, 1782, 1733, + 1514, 1173, 1723, 1419, 576, 100, 60, 577, 2506, 2839, + 3240, 2532, 3379, 2775, 2776, 3237, 3238, 2509, 2172, 3306, + 3307, 2586, 1715, 3301, 2242, 3180, 2176, 2157, 2777, 2250, + 3140, 2880, 2173, 2757, 2243, 3233, 1794, 2244, 3234, 3001, + 2245, 1760, 1785, 2510, 3308, 2177, 1761, 2505, 2840, 1703, + 2246, 3244, 2247, 528, 2761, 578, 560, 561, 827, 1363, + 562, 807, 579, 738, 1791, 580, 581, 2224, 3048, 2610, + 3049, 2305, 2218, 1428, 2299, 1841, 1763, 1429, 518, 1855, + 2611, 2560, 1842, 582, 1046, 68, 69, 960, 70, 2995, + 71, 72, 1680, 1681, 1682, 665, 677, 678, 2089, 1484, + 1925, 670, 1117, 1649, 652, 653, 2214, 692, 1755, 1644, + 1645, 2102, 2473, 1673, 1674, 1126, 1127, 1913, 3208, 1914, + 1915, 1477, 1478, 3087, 1661, 1665, 1666, 2123, 2113, 1652, + 2374, 2918, 2919, 2920, 2921, 2922, 2923, 2924, 1047, 2649, + 3098, 1669, 1670, 1129, 1130, 1131, 1678, 2133, 74, 75, + 2074, 2449, 2450, 624, 2935, 1503, 1683, 2653, 2654, 2655, + 2939, 2940, 2941, 625, 955, 956, 979, 974, 1492, 1934, + 626, 627, 1890, 1891, 2343, 981, 1927, 1945, 1946, 2661, + 1811, 808, 2158, 1519, 1366, 810, 1048, 811, 1341, 1049, + 1345, 813, 1050, 1051, 1052, 816, 1053, 1054, 1055, 819, + 1337, 1056, 1057, 1356, 1385, 1386, 1387, 1388, 1389, 1390, + 1391, 1392, 1393, 1101, 1684, 1059, 1060, 1061, 1062, 1063, + 1064, 629, 1065, 1066, 1604, 2068, 2448, 2928, 3095, 3096, + 2706, 2970, 3122, 3220, 3338, 3366, 3367, 3393, 1067, 1068, + 1550, 1551, 1552, 1968, 1969, 1970, 1971, 2062, 1598, 1599, + 1069, 2842, 1601, 1989, 2931, 2932, 1102, 1470, 1543, 1304, + 1305, 1520, 1444, 1445, 1451, 1865, 1459, 1463, 1895, 1896, + 1471, 2030, 1070, 1964, 1965, 2399, 1528, 1071, 1185, 1555, + 2701, 2065, 1602, 2024, 1078, 1072, 1079, 1074, 1539, 1540, + 2410, 2673, 2674, 1995, 2130, 1633, 2135, 2136, 831, 1075, + 1076, 1077, 1306, 504, 822, 3324, 1397, 1107, 1307, 2020, + 583, 987, 1959, 584, 1316, 1774, 585, 3078, 2893, 1329, + 1795, 2255, 529, 586, 587, 513, 81, 82, 83, 701, + 1421, 588, 1422, 1423, 920, 84, 1515, 922, 923, 590, + 659, 660, 1516, 1618, 1517, 591, 635, 1513, 592, 1097, + 674, 1098, 1100, 593, 1091, 2462, 2091, 89, 90, 91, + 107, 1183, 594, 646, 647, 595, 94, 1141, 648, 666, + 596, 597, 3074, 598, 2589, 1323, 514, 506, 507, 824, + 1263, 1309, 1264 }; /* YYPACT[STATE-NUM] -- Index in YYTABLE of the portion describing STATE-NUM. */ -#define YYPACT_NINF -2936 +#define YYPACT_NINF -2957 static const int yypact[] = { - 6190, -63, 907, -2936, -2936, 254, -63, 47290, 61705, -63, - 95, 1717, 49214, -2936, 273, 6594, -63, 52100, 68335, 526, - 233, 30186, 659, 52581, 52581, -2936, -2936, -2936, 61705, 52100, - 53062, -63, 694, 62186, -2936, -63, 32110, 49695, 444, -2936, - 52100, 20, 364, 53543, 52100, 3906, 908, 384, -2936, -2936, - -2936, -2936, -2936, 108, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, 144, -2936, 50, - 149, 30186, 30186, 1461, 119, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, 468, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, 31629, -2936, -2936, -2936, -2936, -2936, - -2936, 54024, 52100, 54505, 50176, 54986, -2936, 946, -2936, 172, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, 181, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, 198, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, 209, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, 548, -2936, - 473, -2936, 210, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, 3906, 52100, 841, 874, 642, 55467, - -2936, -2936, 52100, -2936, -2936, 897, 824, -2936, -2936, 50657, - -2936, -2936, -2936, 808, 1041, 784, -2936, -2936, 663, -2936, - 160, -2936, -2936, 689, 677, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, 786, -2936, 65553, -2936, 62667, 55948, 56429, - -2936, 656, 68352, 29222, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, 468, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, 52581, 61705, 52581, 702, - 716, 1007, 728, 30667, 792, 32592, 804, 834, 1010, 848, - 889, 896, 918, 364, 29704, 699, 548, -2936, 56910, 56910, - -45, 944, -2936, 56910, 57391, -2936, 852, -2936, 946, -2936, - -2936, -2936, 1188, -2936, 354, 966, -2936, 57872, 57872, 57872, - 989, 1209, -2936, -2936, -2936, 958, -2936, -2936, 1204, 19241, - 19241, 63148, 63148, 946, 63148, 1018, -2936, -2936, 65, -2936, - -2936, -2936, 1461, 1003, 548, -2936, -2936, 49695, -2936, -2936, - 305, 1325, 19241, 52100, 1005, -2936, 1012, 1005, 1015, 1040, - -2936, 6190, 1377, 1261, 49695, 797, 797, 1491, 797, 906, - 972, 3276, 4549, -2936, 1767, -2936, 1053, -2936, 52100, 1158, - 1090, 1357, -2936, 1003, 1444, 165, 1259, 1454, 6302, 1460, - 819, 1468, 854, 1574, 19241, 44885, 548, -2936, 10658, 1131, - 1135, -2936, -2936, -2936, -2936, 75, 1372, -2936, 1601, -2936, - -2936, 1189, 58353, 58834, 59315, 59796, 1565, -2936, -2936, 1507, - -2936, -2936, -2936, 1194, -2936, -2936, -2936, 176, -2936, -2936, - -2936, -2936, -2936, -2936, 1226, -2936, 1226, 1226, -2936, -2936, - -2936, -2936, 1191, 1191, 1374, 1196, -2936, -2936, -2936, 1550, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - 1200, 937, -2936, 1226, -2936, 1191, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, 67898, -2936, -2936, -2936, -2936, 443, 502, - -2936, 1202, -2936, -2936, 96, 1214, -2936, 1679, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, 1228, -2936, 5523, 1191, - -2936, -2936, 1582, 1231, 100, -2936, 1592, 116, -2936, 1594, - 1473, 13694, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, 364, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, 401, -2936, -2936, - 42146, -2936, -2936, 68352, 1332, 1250, -2936, 1590, 19241, 19241, - 1272, 2503, 56910, 57391, 19241, 52100, -2936, 19241, 23795, 1275, - 19241, 19241, 11164, 19241, 28260, 56910, 944, 1258, -2936, 688, - 52100, 1277, -2936, 1354, 1354, 694, 30186, 1567, -2936, 924, - 1559, 1493, -2936, 30186, 1493, 1430, 1286, 1569, 1493, -2936, - 685, 1572, 1354, 33073, 1292, -2936, 1354, 1513, -2936, -2936, - 52581, 42146, 13694, 66019, 1756, -2936, -2936, -2936, -2936, 1573, - 61705, 1304, -2936, -2936, -2936, -2936, -2936, -2936, 428, 1803, - 167, 1807, 19241, 167, 167, 1315, 215, 215, -2936, 1316, - -2936, 216, 1317, 1318, 1815, 1818, 197, 153, 937, 167, - 19241, -2936, 215, 1326, 1822, 1331, 1827, 163, 190, -2936, - 217, 19241, 19241, 19241, 1698, 19241, 10152, 1834, 44885, -2936, - 52100, 634, -2936, 548, 1346, 946, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, 204, 7062, -2936, -2936, -2936, -2936, - 1393, -2936, -2936, -2936, -2936, 1586, 19241, -2936, -2936, 1353, - 1567, -2936, 218, -2936, -2936, 1567, -2936, -2936, -2936, -2936, - -2936, 230, 1765, 41184, 41665, 61705, 548, 63629, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, 732, -2936, 468, 43699, 1376, - 1385, 1005, 52100, 52100, 1851, -2936, -2936, -2936, 49695, 186, - 1678, 1483, -2936, -2936, 1461, 1461, 14687, 982, 173, 719, - 15193, 19747, 1731, 1618, 192, 599, 1733, -2936, 1622, 1842, - 23795, 19241, 19241, 906, 972, 19241, 1012, -2936, -2936, -2936, - 1673, 52100, 47771, 898, 1009, 1404, 1489, 1411, 19, 1824, - -2936, 1409, -2936, 1501, 52100, 67898, 234, -2936, 1859, 234, - 234, 525, 1860, 1505, 288, 1667, 55, -79, 1409, 2438, - -2936, 49695, 114, 115, 1409, 52100, 1511, 619, 1409, 61705, - 1250, 7252, 1418, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, 169, 13694, -2936, 1057, 1103, 1266, 419, - 187, 1284, 1313, 13694, 1351, 1388, 171, 1520, 1525, 1564, - 1575, 1581, 1605, 1611, 1616, 156, 1624, 1637, 1644, 1650, - 1652, 1654, -2936, 1656, 185, 1681, 205, 13694, 1693, -2936, - 43699, -12, -2936, -2936, 1695, 203, -2936, 1514, 61705, 1469, - 52100, 998, 1784, 1835, 66485, 52100, 1669, 2438, 1670, 1439, - 1901, 1680, 1135, 1683, 1446, -2936, 64110, 1945, -2936, -2936, - -2936, -2936, -2936, -2936, 1451, -2936, -2936, 19241, -2936, -2936, - -2936, 1947, -2936, 66019, 66019, 1226, 1226, -2936, -2936, 1915, - 1543, 1544, 1947, -2936, 1947, -2936, 61705, -2936, -2936, 66019, - -2936, 61705, 1462, 1465, 1947, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, 1947, 1552, -2936, 1553, 1557, 1558, -2936, -2936, -2936, - -2936, -2936, 61705, 44885, 1457, 61705, -2936, 52100, 52100, -2936, - 52100, 61705, -2936, 743, 43699, 1962, 68352, 46328, -2936, -2936, - -2936, -2936, 973, 1000, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, 946, 44885, -2936, 2087, 52581, 42744, 1475, - 19241, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, 1476, 1813, -2936, -2936, 6684, 1481, 42784, 1486, 23795, - 23795, 548, 1790, -2936, -2936, 23795, 1487, 46809, 42664, 1490, - 1495, 43125, 15699, 19241, 15699, 15699, 43165, -2936, 1496, 43205, - 56910, 1498, 52100, 51138, -2936, -2936, -2936, 19241, 19241, 944, - 51619, 1541, 30186, -2936, 30186, -2936, 1783, 30186, -2936, -2936, - 3465, -2936, 30186, 1788, 19241, 30186, -2936, 30186, 1729, 1734, - 1509, 30186, -2936, 52100, 1510, 52100, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, 1508, 777, -2936, 802, 1866, -2936, - -2936, 19241, 19241, -2936, 43699, 1561, 19241, -2936, 19241, 43883, - 37404, 16205, 43883, 2000, 2000, 35478, -2936, 1674, 43245, -2936, - 1517, 2562, 7696, 1516, -2936, -2936, 1519, -2936, 1522, 1512, - 39741, 168, 548, 548, 19241, -2936, 19241, 2090, 2090, -2936, - 240, 66019, 19241, 19241, 19241, 19241, 19241, 19241, 19241, 19241, - 44404, 1609, 151, 61705, 19241, 19241, 1524, 1635, -2936, 19241, - 1762, -2936, 1538, 19241, 1623, 297, 19241, 19241, 19241, 19241, - 19241, 19241, 19241, 19241, 19241, -2936, -2936, 26306, 275, 691, - 1864, 1883, -4, 523, 19241, 1882, 10658, -2936, 1882, -2936, - -2936, -2936, -2936, -2936, 219, -2936, -2936, -2936, 1508, -2936, - 1508, -2936, 61705, -2936, 52100, 305, 48733, 19241, -2936, -2936, - 1546, 1555, 505, 1615, -2936, -2936, 52100, 35959, 1847, -2936, - 697, 1566, -2936, 42624, 1821, 1847, 1461, -2936, -2936, 24807, - 1687, 1840, 1799, -2936, -2936, 1779, 1780, -2936, 1585, 43842, - 20253, 20253, -2936, 1360, 43699, 1365, -2936, -2936, -2936, -2936, - -2936, -2936, 679, -2936, 52100, 458, 33554, -2936, 1587, 60, - -2936, 1829, 1895, 1871, 1731, 599, 1589, -2936, -2936, 1008, - 1600, 64591, 52100, 1873, 1845, 1888, -108, 66019, -2936, -2936, - -2936, -2936, 52100, 61705, 60277, 65072, 45366, 52100, 44885, -2936, - -2936, -2936, -2936, 52100, 1129, 52100, 7806, -2936, -2936, -2936, - 234, -2936, -2936, -2936, -2936, -2936, 61705, 52100, -2936, -2936, - 234, 61705, 52100, 234, -2936, 1706, 52100, 52100, 52100, 52100, - 1802, 52100, 52100, -2936, -2936, -2936, 20759, 42, 42, 1826, - 11670, 148, -2936, 19241, 19241, 1792, -2936, -2936, 810, 1837, - 142, -2936, 1664, 52100, 52100, 52100, 52100, 52100, 765, -2936, - -2936, -2936, -2936, -2936, 1619, -2936, 1621, 1975, -2936, 2438, - 1976, 48252, 801, 2891, 1977, 1668, 1987, 12176, 2096, 1870, - -2936, -2936, 1857, -2936, 19241, 1640, 1645, 96, 814, -2936, - -2936, 1651, 1465, 1665, 1671, 1658, 1659, 66019, 816, -2936, - 818, 1947, 137, 1661, 1663, 1471, 112, 544, 1370, 100, - -2936, 44885, -2936, 116, -2936, 1884, 164, -2936, 13694, 19241, - -2936, -2936, -2936, -2936, -2936, -2936, 1250, 26798, -2936, 831, - -2936, -2936, 2112, 946, 2112, 98, -2936, -2936, 2112, -2936, - 2109, 2112, -2936, -2936, 66019, -2936, 7841, -2936, 19241, 19241, - -2936, 19241, 2002, -2936, 2159, 2159, 66019, 23795, 23795, 23795, - 23795, 23795, 23795, 683, 1326, 23795, 23795, 23795, 23795, 23795, - 23795, 23795, 23795, 23795, 25313, 399, -2936, -2936, 838, 2135, - 19241, 19241, 2014, 2002, 19241, -2936, 66019, 1682, -2936, 1684, - 1685, 19241, -2936, 66019, -2936, 52100, 1688, 32, 29, -2936, - 1690, 1691, -2936, 1567, -2936, 949, 999, 52100, 3261, 4445, - 5202, -2936, -2936, 19241, 2004, 3465, 3465, 30186, -2936, 19241, - 1701, -2936, -2936, 30186, 2019, -2936, 3465, -2936, -2936, 34035, - 3465, 66019, 839, -2936, 52100, 66019, 843, 42146, -2936, 13694, - -2936, 66019, -2936, -2936, -2936, -2936, -2936, -2936, 1692, 1689, - 19241, 77, -2936, 7929, 6358, -2936, 1699, -2936, 1675, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, 1702, -2936, 1694, -2936, 1704, 52100, - -2936, 21265, -2936, 61705, -2936, -2936, 19241, 19241, 52100, -2936, - 1698, -2936, 1707, 37893, -2936, -2936, -2936, 268, 330, 5666, - 523, 3507, 3507, 3507, 43883, -2936, -2936, -2936, 1724, -2936, - 23795, 23795, -2936, 5852, 2654, 10152, -2936, -2936, 2015, -2936, - 692, -2936, 1711, -2936, -2936, 3583, -2936, 37404, 43864, 19241, - 159, -2936, 19241, 1524, 19241, 1797, 3507, 3507, 3507, 342, - 342, 268, 268, 268, 330, 523, -2936, -2936, -2936, 1714, - 19241, 44885, -2936, 1716, 1718, 2068, 1331, 19241, -2936, -2936, - 30186, 1541, -12, 1541, 1947, 2090, -2936, 1012, -2936, 1012, - -2936, 43699, 52100, -2936, -2936, 1979, 1720, 30186, 1768, 2181, - 2170, 61705, -2936, -2936, 1735, 1882, 1744, -2936, -2936, 1753, - 19241, 2532, 1753, -2936, 1847, -26, 1958, 1038, 1038, 1360, - 1959, -2936, -2936, 1804, -2936, -2936, -2936, 19241, 12682, 1408, - -2936, 1414, -2936, -2936, -2936, -2936, -2936, 1740, -2936, 2007, - -2936, 52100, -2936, -2936, 23795, 2187, 19241, 34516, 2189, 1993, - -2936, -2936, -2936, 1836, 1409, 19241, 1988, -2936, 184, 1754, - 2108, -71, 2060, 61705, -2936, 412, 588, -2936, 223, 2110, - 164, 2111, 164, 44885, 44885, 44885, 853, -2936, -2936, -2936, - 946, -2936, 562, 855, -2936, -2936, -2936, -2936, 1844, 662, - 1409, 2438, -2936, -2936, -2936, -2936, -2936, -2936, -2936, 175, - 706, 1409, 1846, -2936, 1853, -2936, 1854, 708, 1409, -2936, - -2936, 8630, 43699, 597, 148, 148, 148, 13694, -2936, 1989, - 1998, 1760, 43699, 43699, -2936, 157, -2936, 61705, -2936, -2936, - -2936, 1792, 1835, 52100, 2438, 1775, 2227, 1135, 1446, -2936, - 1937, 705, 140, -2936, 61705, 52100, 52100, 52100, 60758, -2936, - -2936, -2936, 1787, 1785, -2936, -34, 2011, 2010, 52100, 1832, - 52100, 1411, 2255, 52100, -2936, 865, 16711, 2148, 52100, 1796, - -2936, -2936, -2936, -2936, 1947, -2936, -2936, 369, 369, -2936, - -2936, 61705, -2936, 1800, -2936, 1801, -2936, -2936, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, 61705, -2936, - -2936, 43699, -2936, -2936, 45847, -2936, -2936, -2936, -2936, -2936, - 44885, -2936, 946, -2936, 946, 2026, 61705, 40222, 946, 40703, - 946, 1809, -2936, 43699, 37915, 43699, 2014, -2936, 141, 2159, - 252, 252, 252, 5272, 2134, 456, 1805, 252, 252, 252, - 379, 379, 141, 141, 141, 2159, 399, 852, 46809, 1808, - -2936, 43699, 43699, -2936, -2936, 1810, -2936, -2936, -2936, -2936, - 1811, 1814, -2936, -2936, -2936, -2936, -2936, -2936, 61705, 1126, - 1541, 444, 444, 444, 444, -2936, 52100, 52100, 52100, 43699, - 2243, 2124, -2936, -2936, 3465, 43699, 52100, -2936, 27775, -2936, - 52100, -2936, 2157, -2936, 2248, -2936, 52100, 876, -2936, -2936, - -2936, -2936, -2936, 43533, 19241, -2936, 2188, -2936, 19241, 19241, - -2936, 37404, -2936, -2936, 1831, 10152, 43555, -2936, 2137, 37962, - 43699, -2936, 1674, -2936, 2090, 19241, 3631, 3922, 19241, 1833, - 19241, 2166, -2936, -2936, 1848, -2936, -2936, 66019, 19241, 1843, - 3610, 23795, 23795, 4168, -2936, 4589, 19241, 10152, -2936, 39161, - 1825, 1849, 1826, 17217, -2936, 2044, 1852, -2936, 2004, 148, - 2004, 1856, -2936, -2936, -2936, -2936, 1866, -2936, 19241, 2001, - 61705, 517, 2485, 887, -2936, 548, 35959, 1768, 19241, 430, - -2936, -2936, 1858, -2936, 1753, -2936, -2936, -2936, 2064, -2936, - -2936, -2936, 52100, -2936, 1863, -2936, 33554, 2184, 10152, -2936, - 33554, 52100, 52100, 38017, 2221, -2936, 61705, 61705, 61705, -2936, - 61705, 1868, 1872, 738, 1875, 309, -2936, 2510, 738, 2205, - 509, 1411, 288, 4427, 623, -2936, -2936, -2936, 1950, 52100, - -2936, 61705, -2936, -2936, -2936, -2936, -2936, 45366, -2936, -2936, - 36922, 44885, -2936, 44885, 52100, 52100, 52100, 52100, 52100, 52100, - 52100, 52100, 52100, 52100, 19241, -2936, 19241, 1877, 1878, 1879, - 1826, -2936, -2936, -2936, -2936, -2936, -2936, -79, -2936, 157, - 1881, -2936, 48252, 3906, 1668, 2344, 1835, 86, 61224, -2936, - 1885, 1886, 905, 2438, 1889, 2345, -2936, 801, 48252, -2936, - -2936, -2936, 2309, -2936, 656, 188, -2936, 1135, -2936, 3906, - 1446, -2936, 3906, 43699, 61705, 1953, -2936, 1465, 1897, -2936, - -2936, 1465, 66019, 1465, -2936, -2936, 164, 921, -2936, -2936, - -2936, -2936, -2936, 61705, 1896, -2936, 1896, -2936, -2936, 1896, - -2936, -2936, -2936, -2936, 23795, 2232, 1902, 66019, -2936, -2936, - 52100, -2936, -2936, -2936, 927, 1900, 2004, 52100, 52100, 52100, - 52100, -2936, -2936, -2936, 17723, 19241, 1944, -2936, 1905, 14200, - 2214, -2936, 27290, -2936, 220, 1908, 34035, 61705, -2936, -2936, - -2936, 19241, 43699, -2936, 43699, -2936, -2936, -2936, 7109, -2936, - 1906, 1911, 61705, 19241, -2936, -2936, -2936, 779, 19241, 19241, - 5852, -2936, 43929, 19241, 66019, 931, 5852, 321, 19241, 5125, - 5236, 19241, 19241, 4723, 38207, -2936, 21771, 13188, -2936, 1914, - 19241, 38516, 36440, -2936, 30186, 2124, 1921, 2124, 946, 1922, - 43699, 19241, -2936, -2936, -2936, -2936, 1968, -80, 32110, 2144, - -2936, 1938, 61705, -2936, 2001, 43699, -2936, -2936, 37404, -2936, - -2936, -2936, -2936, -2936, 2373, 1676, 1933, 1934, -2936, 1297, - -2936, -2936, 61705, 1940, -2936, 1942, 738, -2936, 61705, 1982, - -2936, 212, 2240, 74, -2936, 19241, -2936, 2332, 2406, 2510, - 1949, 61705, 52100, 23795, -2936, 707, 162, -2936, 2229, 52100, - 1982, 2366, -2936, -2936, -2936, 309, -2936, 2267, 2182, -2936, - 234, -2936, 19241, 309, 2190, 260, 61705, -2936, -2936, 3128, - -2936, 66019, 164, 164, -2936, 1957, 1964, 1965, 1966, 1967, - 1970, 1978, 1980, 1981, 1983, 1984, -2936, 1985, 1991, 1994, - 1995, 1997, 1999, 2003, 2008, 1200, 2009, -2936, 2012, 1858, - 2013, 2017, 2023, 2025, 2027, 66951, 2028, 2029, 2031, 2032, - 1202, 2033, 973, 1000, -2936, -2936, -2936, -2936, -2936, -2936, - 1231, 2034, -2936, 1971, -2936, -2936, -2936, 2054, -2936, 2063, - -2936, -2936, -2936, -2936, -2936, 1996, 2005, -2936, -2936, -2936, - 148, 1250, 117, 61705, 2016, 1832, 2444, 18229, 837, 2233, - 2021, -2936, 946, 1668, -2936, 48252, 3035, 222, 2010, -2936, - 170, 1832, -2936, 2408, 1668, 2066, 2476, -2936, 2256, 61705, - 2041, -2936, -2936, -2936, -2936, 45847, 1896, 5318, 23795, 66019, - 932, 943, -2936, 2534, 2202, 2124, -2936, -2936, -2936, -2936, - -2936, 2045, 325, 2046, 9646, 2050, -2936, -2936, -2936, -2936, - -2936, -2936, 43699, 43699, 61705, 2223, -2936, -2936, 2052, 2049, - 34997, 2498, 2057, -2936, 13694, -2936, 2367, -2936, 28741, -2936, - 43699, 19241, -2936, -2936, 39208, 2376, 5852, 5852, 43929, 945, - -2936, 5852, 19241, 19241, 5852, 5852, 19241, -2936, 9138, 669, - -2936, 950, -2936, 38538, -2936, 67417, -2936, -2936, 1944, 946, - 1944, -2936, -2936, 2055, -2936, -2936, -2936, 2125, -2936, -2936, - 960, 2477, 2001, 19241, -2936, -2936, 2077, 33554, -2936, -2936, - -2936, -2936, 33554, 738, -2936, 2252, 1982, 2083, -2936, -2936, - -2936, -2936, -2936, -2936, 38585, -2936, 43, 19241, -2936, 965, - 5272, -2936, -2936, -2936, -2936, 1982, 1135, -2936, 52100, 2557, - 2454, -2936, -2936, 43699, -2936, -2936, 1947, 1947, -2936, -2936, - 2248, -2936, -2936, -2936, -2936, 1231, 606, 36922, 52100, 52100, - -2936, -2936, 2102, -2936, -2936, -79, 2484, 964, 801, -2936, - 3906, 3906, 43699, 52100, 2458, 48252, -2936, 2567, 2113, 52100, - 1832, 375, 375, -2936, 2258, -2936, 2259, -2936, -2936, 2572, - 500, -2936, 18735, 52100, -2936, -2936, 31148, -2936, 5318, 977, - -2936, -2936, 2117, 2122, -2936, 1944, 19241, 2126, 19241, -2936, - 22277, 2573, 2115, -2936, 19241, 2191, 25819, -2936, 19241, -2936, - 52100, 56910, 42, 2130, 56910, -2936, -2936, -2936, 19241, -2936, - 5852, 5852, 5852, 19241, -2936, 19241, -2936, -2936, -2936, 2323, - 2223, -2936, 2223, 19241, 3906, 548, 3921, 61705, -21, -2936, - 43699, -2936, -2936, -2936, 52100, -2936, 44885, -2936, 738, -50, - 2131, 19241, 38640, 2360, -2936, -2936, 2392, -2936, 2451, -2936, - 2197, 676, 2213, -2936, -2936, -2936, -2936, 1250, 946, 1668, - 2010, 2066, -2936, 2141, 52100, 3906, 801, 656, -2936, -2936, - -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, -2936, - -2936, 3906, 2571, 2359, 2574, 3906, 43699, 1953, 19241, 73, - -2936, 991, 2569, -2936, -2936, 2640, 2223, 2150, 22277, 2152, - -2936, 2155, 61705, 43699, 2297, -2936, -2936, 2172, -2936, -2936, - 148, 19241, -2936, 39263, 2168, 2176, 2620, 1826, 2191, 2191, - -2936, -80, -2936, -2936, 2589, 31148, 2553, 1135, 738, 2199, - 992, -2936, -2936, -2936, -2936, -2936, 2438, -2936, 38830, 2429, - 467, 2413, 2131, 19241, -2936, 2269, -2936, -2936, -2936, 2658, - -2936, -2936, 48252, 2201, 2066, 2010, 1832, 2066, 2414, -2936, - 2421, -2936, 2208, 39139, 61705, 61705, 1668, 31148, 61705, 2209, - 2191, -2936, 2211, -2936, -2936, -2936, 51138, -2936, 2215, 2216, - -2936, -2936, -2936, 19241, 41, -2936, -2936, 2262, 52100, 1006, - 44, 2392, 36922, -2936, 44885, 1466, -50, 2515, -2936, -2936, - -2936, -2936, 127, 2433, -2936, 2436, -2936, 43699, -2936, 3906, - 48252, -2936, -2936, -2936, -2936, -2936, -2936, 31148, 2569, -2936, - 697, -2936, 1541, -2936, 697, -2936, -2936, -2936, -2936, 946, - -2936, 1490, 22783, 22783, 2220, 3906, -2936, 1541, -2936, 2347, - 2413, -2936, -2936, -2936, -2936, -2936, 239, 239, 2605, -2936, - 2286, -2936, 2066, 1027, 61705, 1753, -2936, 1753, 24301, 2370, - 143, 42704, -2936, -2936, -2936, -2936, -2936, -2936, 35959, -2936, - -2936, 2702, -2936, 178, -2936, -2936, -2936, 1668, 697, -2936, - -2936, 2693, -2936, -2936, -2936, -2936, -2936, 1541, 738, -2936, - -2936, -2936, 1541, 1753, 23289, -2936, -2936, -2936, -2936 + 7643, -46, 769, -2957, -2957, 322, -46, 47674, 62179, -46, + 106, 1903, 49610, -2957, 343, 4780, -46, 52514, 68851, 363, + 245, 30410, 480, 52998, 52998, -2957, -2957, -2957, 62179, 52514, + 53482, -46, 729, 62663, -2957, -46, 32346, 50094, -15, -2957, + 52514, 51, 104, 53966, 52514, 3461, 840, 344, -2957, -2957, + -2957, -2957, -2957, 108, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, 154, -2957, 100, + 179, 30410, 30410, 851, 378, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, 415, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, 31862, -2957, -2957, -2957, -2957, -2957, + -2957, 54450, 52514, 54934, 50578, 55418, -2957, 634, -2957, 172, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, 215, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, 217, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, 222, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, 586, -2957, 394, -2957, 224, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, 3461, 52514, 492, + 536, 577, 55902, -2957, -2957, 52514, -2957, -2957, 758, 814, + -2957, -2957, 51062, -2957, -2957, -2957, 866, 1062, 827, -2957, + -2957, 790, -2957, 169, -2957, -2957, 876, 740, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, 880, -2957, 66051, -2957, + 63147, 56386, 56870, -2957, 775, 68868, 29440, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, 415, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, 52998, + 62179, 52998, 794, 804, 1153, 824, 30894, 838, 32831, 846, + 848, 1122, 855, 861, 863, 868, 104, 29925, 847, 586, + -2957, 57354, 57354, -65, 2151, -2957, 57354, 57838, -2957, 884, + -2957, 634, -2957, -2957, -2957, 1227, -2957, 502, 897, -2957, + 58322, 58322, 58322, 927, 1196, -2957, -2957, -2957, 929, -2957, + -2957, 1158, 19399, 19399, 63631, 63631, 634, 63631, 969, -2957, + -2957, 57, -2957, -2957, -2957, 851, 952, 586, -2957, -2957, + 50094, -2957, -2957, 300, 1302, 19399, 52514, 968, -2957, 974, + 968, 981, 992, -2957, 7643, 1325, 1219, 50094, 730, 730, + 1455, 730, 830, 839, 4751, 4018, -2957, 1511, -2957, 1006, + -2957, 52514, 1111, 1035, 1307, -2957, 952, 1383, 870, 1195, + 1407, 3469, 1425, 924, 1449, 1157, 1570, 19399, 45254, 586, + -2957, 10765, 1113, 1112, -2957, -2957, -2957, -2957, 773, 1351, + -2957, 1586, -2957, -2957, 1181, 58806, 59290, 59774, 60258, 1560, + -2957, -2957, 1505, -2957, -2957, -2957, 1194, -2957, -2957, -2957, + 184, -2957, -2957, -2957, -2957, -2957, -2957, 1213, -2957, 1213, + 1213, -2957, -2957, -2957, -2957, 1166, 1166, 1366, 1184, -2957, + -2957, -2957, 1542, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, 1189, 554, -2957, 1213, -2957, 1166, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, 68411, -2957, -2957, -2957, + -2957, 640, 742, -2957, 1191, -2957, -2957, 133, 1205, -2957, + 1658, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, 1215, + -2957, 3392, 1166, -2957, -2957, 1574, 1221, 165, -2957, 1587, + 183, -2957, 1593, 1460, 13819, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, 104, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + 728, -2957, -2957, 42619, -2957, -2957, 68868, 1326, 1244, -2957, + 1584, 19399, 19399, 1252, 2029, 57354, 57838, 19399, 52514, -2957, + 19399, 23980, 1263, 19399, 19399, 11274, 19399, 28472, 57354, 2151, + 1268, -2957, 719, 52514, 1271, -2957, 1377, 1377, 729, 30410, + 1573, -2957, 1100, 1569, 1501, -2957, 30410, 1501, 1740, 1292, + 1577, 1501, -2957, 763, 1592, 1377, 33315, 1295, -2957, 1377, + 1512, -2957, -2957, 52998, 42619, 13819, 66520, 1764, -2957, -2957, + -2957, -2957, 1585, 62179, 1310, -2957, -2957, -2957, -2957, -2957, + -2957, 684, 1810, 167, 1813, 19399, 167, 167, 1316, 227, + 227, -2957, 1317, -2957, 228, 1318, 1323, 1822, 1823, 192, + 160, 554, 167, 19399, -2957, 227, 1327, 1824, 1328, 1829, + 190, 198, -2957, 229, 19399, 19399, 19399, 1691, 19399, 10256, + 1827, 45254, -2957, 52514, 603, -2957, 586, 1334, 634, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, 206, 6802, -2957, + -2957, -2957, -2957, 1373, -2957, -2957, -2957, -2957, 1543, 19399, + -2957, -2957, 1333, 1573, -2957, 230, -2957, -2957, 1573, -2957, + -2957, -2957, -2957, -2957, 259, 1741, 41651, 42135, 62179, 586, + 64115, -2957, -2957, -2957, -2957, -2957, -2957, -2957, 772, -2957, + 415, 44174, 1338, 1340, 968, 52514, 52514, 1806, -2957, -2957, + -2957, 50094, 163, 1633, 1472, -2957, -2957, 851, 851, 14818, + 699, 588, 972, 15327, 19908, 1689, 1572, 239, 593, 1697, + -2957, 1575, 1799, 23980, 19399, 19399, 830, 839, 19399, 974, + -2957, -2957, -2957, 1638, 52514, 48158, 964, 979, 1357, 1451, + 1380, 54, 1798, -2957, 1388, -2957, 1479, 52514, 68411, 212, + -2957, 1842, 212, 212, 735, 1844, 1487, 389, 1648, 58, + -82, 1388, 2602, -2957, 50094, 247, 68, 1388, 52514, 1492, + 71, 1388, 62179, 1244, 39293, 1397, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, 178, 13819, -2957, 1022, + 1140, 1150, 410, 149, 1226, 1249, 13819, 1260, 1332, 185, + 1365, 1408, 1414, 1435, 1450, 1481, 1488, 1540, 138, 1578, + 1629, 1647, 1652, 1668, 1672, -2957, 1692, 195, 1696, 174, + 13819, 1698, -2957, 44174, 4, -2957, -2957, 1707, 200, -2957, + 1495, 62179, 1452, 52514, 1024, 1767, 1826, 66989, 52514, 1653, + 2602, 1654, 1417, 1886, 1660, 1112, 1663, 1430, -2957, 64599, + 1932, -2957, -2957, -2957, -2957, -2957, -2957, 1433, -2957, -2957, + 19399, -2957, -2957, -2957, 1929, -2957, 66520, 66520, 1213, 1213, + -2957, -2957, 1900, 1523, 1532, 1929, -2957, 1929, -2957, 62179, + -2957, -2957, 66520, -2957, 62179, 1456, 1457, 1929, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, 1929, 1531, -2957, 1534, 1548, 1550, + -2957, -2957, -2957, -2957, -2957, 62179, 45254, 1446, 62179, -2957, + 52514, 52514, -2957, 52514, 62179, -2957, 791, 44174, 1946, 68868, + 46706, -2957, -2957, -2957, -2957, 578, 959, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, 634, 45254, -2957, 3799, + 52998, 43220, 1467, 19399, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, 1468, 1807, -2957, -2957, 5988, 1469, + 43267, 1473, 23980, 23980, 586, 271, -2957, -2957, 23980, 1474, + 47190, 43140, 1476, 1493, 43508, 15836, 19399, 15836, 15836, 43645, + -2957, 1494, 43769, 57354, 1477, 52514, 51546, -2957, -2957, -2957, + 19399, 19399, 2151, 52030, 1514, 30410, -2957, 30410, -2957, 1771, + 30410, -2957, -2957, 4015, -2957, 30410, 1779, 19399, 30410, -2957, + 30410, 1725, 1726, 1499, 30410, -2957, 52514, 1502, 52514, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, 1497, 801, -2957, + 805, 873, -2957, -2957, 19399, 19399, -2957, 44174, 1545, 19399, + -2957, 19399, 6854, 37673, 16345, 6854, 1996, 1996, 35735, -2957, + 1664, 43889, -2957, 1508, 87, 8057, 1504, -2957, -2957, 1500, + -2957, 1507, 1503, 40199, 181, 586, 586, 19399, -2957, 19399, + 4151, 4151, -2957, 193, 66520, 19399, 19399, 19399, 19399, 19399, + 19399, 19399, 19399, 44770, 1603, 173, 62179, 19399, 19399, 1520, + 152, -2957, 19399, 1753, -2957, 1524, 19399, 1610, 828, 19399, + 19399, 19399, 19399, 19399, 19399, 19399, 19399, 19399, -2957, -2957, + 26506, 207, 639, 1854, 1873, -2, 393, 19399, 1866, 10765, + -2957, 1866, -2957, -2957, -2957, -2957, -2957, 231, -2957, -2957, + -2957, 1497, -2957, 1497, -2957, 62179, -2957, 52514, 300, 49126, + 19399, -2957, -2957, 1529, 1538, 168, 1598, -2957, -2957, 52514, + 36219, 1831, -2957, 324, 1539, -2957, 43100, 1788, 1831, 851, + -2957, -2957, 24998, 1669, 1828, 1768, -2957, -2957, 1742, 1747, + -2957, 1547, 44292, 20417, 20417, -2957, 1025, 44174, 1371, -2957, + -2957, -2957, -2957, -2957, -2957, 657, -2957, 52514, 535, 33799, + -2957, 1552, 84, -2957, 2134, 1879, 1855, 1689, 593, 1580, + -2957, -2957, 1285, 1579, 65083, 52514, 1856, 1809, 1863, 298, + 66520, -2957, -2957, -2957, -2957, 52514, 62179, 60742, 65567, 45738, + 52514, 45254, -2957, -2957, -2957, -2957, 52514, 988, 52514, 3668, + -2957, -2957, -2957, 212, -2957, -2957, -2957, -2957, -2957, 62179, + 52514, -2957, -2957, 212, 62179, 52514, 212, -2957, 1337, 52514, + 52514, 52514, 52514, 1681, 52514, 52514, -2957, -2957, -2957, 20926, + 63, 63, 1797, 11783, 134, -2957, 19399, 19399, 1761, -2957, + -2957, 843, 1811, 176, -2957, 1636, 52514, 52514, 52514, 52514, + 52514, 1491, -2957, -2957, -2957, -2957, -2957, 1594, -2957, 1596, + 1954, -2957, 2602, 1956, 48642, 787, 2022, 1959, 1643, 1961, + 12292, 2070, 1848, -2957, -2957, 1834, -2957, 19399, 1608, 1615, + 133, 849, -2957, -2957, 1620, 1457, 1639, 1640, 1618, 1623, + 66520, 853, -2957, 886, 1929, 151, 1624, 1627, 1475, 709, + 504, 1372, 165, -2957, 45254, -2957, 183, -2957, 1841, 213, + -2957, 13819, 19399, -2957, -2957, -2957, -2957, -2957, -2957, 1244, + 27001, -2957, 888, -2957, -2957, 2093, 634, 2093, 546, -2957, + -2957, 2093, -2957, 2079, 2093, -2957, -2957, 66520, -2957, 38165, + -2957, 19399, 19399, -2957, 19399, 1976, -2957, 2135, 2135, 66520, + 23980, 23980, 23980, 23980, 23980, 23980, 218, 1327, 23980, 23980, + 23980, 23980, 23980, 23980, 23980, 23980, 23980, 25507, 476, -2957, + -2957, 894, 2107, 19399, 19399, 1994, 1976, 19399, -2957, 66520, + 1659, -2957, 1661, 1676, 19399, -2957, 66520, -2957, 52514, 1677, + 41, 36, -2957, 1665, 1670, -2957, 1573, -2957, 712, 871, + 52514, 4027, 4538, 5484, -2957, -2957, 19399, 1981, 4015, 4015, + 30410, -2957, 19399, 1682, -2957, -2957, 30410, 2002, -2957, 4015, + -2957, -2957, 34283, 4015, 66520, 900, -2957, 52514, 66520, 901, + 42619, -2957, 13819, -2957, 66520, -2957, -2957, -2957, -2957, -2957, + -2957, 1679, 1684, 19399, 140, -2957, 38205, 5552, -2957, 1683, + -2957, 1675, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, 1662, -2957, 1690, + -2957, 1694, 52514, -2957, 21435, -2957, 62179, -2957, -2957, 19399, + 19399, 52514, -2957, 1691, -2957, 1702, 38245, -2957, -2957, -2957, + 139, 355, 6232, 393, 2976, 2976, 2976, 6854, -2957, -2957, + -2957, 1709, -2957, 23980, 23980, -2957, 4053, 4145, 10256, -2957, + -2957, 2017, -2957, 793, -2957, 1693, -2957, -2957, 4720, -2957, + 37673, 8105, 19399, 180, -2957, 19399, 1520, 19399, 1782, 2976, + 2976, 2976, 286, 286, 139, 139, 139, 355, 393, -2957, + -2957, -2957, 1706, 19399, 45254, -2957, 1713, 1714, 2051, 1328, + 19399, -2957, -2957, 30410, 1514, 4, 1514, 1929, 4151, -2957, + 974, -2957, 974, -2957, 44174, 52514, -2957, -2957, 1967, 1715, + 30410, 1757, 2183, 2169, 62179, -2957, -2957, 1730, 1866, 1749, + -2957, -2957, 1754, 19399, 3097, 1754, -2957, 1831, -14, 1960, + 1047, 1047, 1025, 1964, -2957, -2957, 1803, -2957, -2957, -2957, + 19399, 12801, 1379, -2957, 1395, -2957, -2957, -2957, -2957, -2957, + 1744, -2957, 2010, -2957, 52514, -2957, -2957, 23980, 2194, 19399, + 34767, 2195, 2000, -2957, -2957, -2957, 1836, 1388, 19399, 1992, + -2957, 117, 1758, 2114, 349, 2065, 62179, -2957, 313, 323, + -2957, 771, 2116, 213, 2117, 213, 45254, 45254, 45254, 936, + -2957, -2957, -2957, 634, -2957, 403, 975, -2957, -2957, -2957, + -2957, 1849, 101, 1388, 2602, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, 294, 658, 1388, 1852, -2957, 1853, -2957, 1857, + 784, 1388, -2957, -2957, 8725, 44174, 609, 134, 134, 134, + 13819, -2957, 1998, 2008, 1780, 44174, 44174, -2957, 170, -2957, + 62179, -2957, -2957, -2957, 1761, 1826, 52514, 2602, 1784, 2248, + 1112, 1430, -2957, 1944, 695, 739, -2957, 62179, 52514, 52514, + 52514, 61226, -2957, -2957, -2957, 1787, 1785, -2957, 23, 2012, + 2011, 52514, 1830, 52514, 1380, 2258, 52514, -2957, 980, 16854, + 2152, 52514, 1795, -2957, -2957, -2957, -2957, 1929, -2957, -2957, + 575, 575, -2957, -2957, 62179, -2957, 1800, -2957, 1808, -2957, + -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, 62179, -2957, -2957, 44174, -2957, -2957, 46222, -2957, -2957, + -2957, -2957, -2957, 45254, -2957, 634, -2957, 634, 2028, 62179, + 40683, 634, 41167, 634, 1805, -2957, 44174, 38285, 44174, 1994, + -2957, 148, 2135, 1680, 1680, 1680, 3395, 2146, 242, 1812, + 1680, 1680, 1680, 334, 334, 148, 148, 148, 2135, 476, + 884, 47190, 1815, -2957, 44174, 44174, -2957, -2957, 1816, -2957, + -2957, -2957, -2957, 1817, 1818, -2957, -2957, -2957, -2957, -2957, + -2957, 62179, 1079, 1514, -15, -15, -15, -15, -2957, 52514, + 52514, 52514, 44174, 2265, 2145, -2957, -2957, 4015, 44174, 52514, + -2957, 27984, -2957, 52514, -2957, 2165, -2957, 2254, -2957, 52514, + 982, -2957, -2957, -2957, -2957, -2957, 43936, 19399, -2957, 2196, + -2957, 19399, 19399, -2957, 37673, -2957, -2957, 1832, 10256, 44056, + -2957, 2143, 38325, 44174, -2957, 1664, -2957, 4151, 19399, 1180, + 2053, 19399, 1837, 19399, 2173, -2957, -2957, 1840, -2957, -2957, + 66520, 19399, 1843, 4948, 23980, 23980, 5586, -2957, 5658, 19399, + 10256, -2957, 39333, 1835, 1846, 1797, 17363, -2957, 2050, 1838, + -2957, 1981, 134, 1981, 1858, -2957, -2957, -2957, -2957, 873, + -2957, 19399, 1999, 62179, 601, 1655, 999, -2957, 586, 36219, + 1757, 19399, 209, -2957, -2957, 1850, -2957, 1754, -2957, -2957, + -2957, 2068, -2957, -2957, -2957, 52514, -2957, 1859, -2957, 33799, + 2180, 10256, -2957, 33799, 52514, 52514, 38669, 2210, -2957, 62179, + 62179, 62179, -2957, 62179, 1860, 1861, 649, 1867, 815, -2957, + 1645, 649, 2188, 211, 1380, 389, 2734, 648, -2957, -2957, + -2957, 1931, 52514, -2957, 62179, -2957, -2957, -2957, -2957, -2957, + 45738, -2957, -2957, 37188, 45254, -2957, 45254, 52514, 52514, 52514, + 52514, 52514, 52514, 52514, 52514, 52514, 52514, 19399, -2957, 19399, + 1862, 1865, 1868, 1797, -2957, -2957, -2957, -2957, -2957, -2957, + -82, -2957, 170, 1870, -2957, 48642, 3461, 1643, 2320, 1826, + 842, 61695, -2957, 1871, 1869, 1004, 2602, 1877, 2324, -2957, + 787, 48642, -2957, -2957, -2957, 2299, -2957, 775, 232, -2957, + 1112, -2957, 3461, 1430, -2957, 3461, 44174, 62179, 1939, -2957, + 1457, 1885, -2957, -2957, 1457, 66520, 1457, -2957, -2957, 213, + 1009, -2957, -2957, -2957, -2957, -2957, 62179, 1881, -2957, 1881, + -2957, -2957, 1881, -2957, -2957, -2957, -2957, 23980, 2222, 1888, + 66520, -2957, -2957, 52514, -2957, -2957, -2957, 1040, 1889, 1981, + 52514, 52514, 52514, 52514, -2957, -2957, -2957, 17872, 19399, 1934, + -2957, 1895, 14328, 2206, -2957, 27496, -2957, 234, 1899, 34283, + 62179, -2957, -2957, -2957, 19399, 44174, -2957, 44174, -2957, -2957, + -2957, 7985, -2957, 1897, 1901, 62179, 19399, -2957, -2957, -2957, + 789, 19399, 19399, 4053, -2957, 6564, 19399, 66520, 1044, 4053, + 210, 19399, 3055, 3286, 19399, 19399, 6608, 38709, -2957, 21944, + 13310, -2957, 1902, 19399, 38749, 36703, -2957, 30410, 2145, 1904, + 2145, 634, 1905, 44174, 19399, -2957, -2957, -2957, -2957, 1971, + -25, 32346, 2124, -2957, 1918, 62179, -2957, 1999, 44174, -2957, + -2957, 37673, -2957, -2957, -2957, -2957, -2957, 2356, 1699, 1914, + 1921, -2957, 1312, -2957, -2957, 62179, 1922, -2957, 1926, 649, + -2957, 62179, 1966, -2957, 310, 2235, 97, -2957, 19399, -2957, + 2323, 2397, 1645, 1935, 62179, 52514, 23980, -2957, 474, 254, + -2957, 2218, 52514, 1966, 2358, -2957, -2957, -2957, 815, -2957, + 2255, 2172, -2957, 212, -2957, 19399, 815, 2175, 244, 62179, + -2957, -2957, 2926, -2957, 66520, 213, 213, -2957, 1941, 1947, + 1948, 1951, 1955, 1962, 1965, 1969, 1973, 1978, 1984, -2957, + 1985, 1986, 1987, 1988, 1990, 1991, 1993, 1995, 1189, 2001, + -2957, 2003, 1850, 2004, 2005, 2009, 2013, 2014, 67458, 2015, + 2016, 2019, 2023, 1191, 2026, 578, 959, -2957, -2957, -2957, + -2957, -2957, -2957, 1221, 2027, -2957, 1957, -2957, -2957, -2957, + 2032, -2957, 2040, -2957, -2957, -2957, -2957, -2957, 1979, 1997, + -2957, -2957, -2957, 134, 1244, 158, 62179, 1982, 1830, 2429, + 18381, 820, 2229, 2030, -2957, 634, 1643, -2957, 48642, 2447, + 786, 2011, -2957, 221, 1830, -2957, 2396, 1643, 2039, 2467, + -2957, 2234, 62179, 2031, -2957, -2957, -2957, -2957, 46222, 1881, + 4794, 23980, 66520, 1070, 1074, -2957, 2526, 2193, 2145, -2957, + -2957, -2957, -2957, -2957, 2035, 80, 2036, 9747, 2034, -2957, + -2957, -2957, -2957, -2957, -2957, 44174, 44174, 62179, 2225, -2957, + -2957, 2047, 2042, 35251, 2496, 2052, -2957, 13819, -2957, 2363, + -2957, 28956, -2957, 44174, 19399, -2957, -2957, 39677, 2372, 4053, + 4053, 6564, 1094, -2957, 4053, 19399, 19399, 4053, 4053, 19399, + -2957, 9236, 622, -2957, 1099, -2957, 38789, -2957, 67927, -2957, + -2957, 1934, 634, 1934, -2957, -2957, 2055, -2957, -2957, -2957, + 2113, -2957, -2957, 1104, 2475, 1999, 19399, -2957, -2957, 2061, + 33799, -2957, -2957, -2957, -2957, 33799, 649, -2957, 2230, 1966, + 2063, -2957, -2957, -2957, -2957, -2957, -2957, 38829, -2957, 72, + 19399, -2957, 182, 3395, -2957, -2957, -2957, -2957, 1966, 1112, + -2957, 52514, 2529, 2422, -2957, -2957, 44174, -2957, -2957, 1929, + 1929, -2957, -2957, 2254, -2957, -2957, -2957, -2957, 1221, 488, + 37188, 52514, 52514, -2957, -2957, 2067, -2957, -2957, -82, 2452, + 1127, 787, -2957, 3461, 3461, 44174, 52514, 2428, 48642, -2957, + 2535, 2073, 52514, 1830, 1695, 1695, -2957, 2219, -2957, 2221, + -2957, -2957, 2541, 376, -2957, 18890, 52514, -2957, -2957, 31378, + -2957, 4794, 1133, -2957, -2957, 2081, 2086, -2957, 1934, 19399, + 2087, 19399, -2957, 22453, 2553, 2088, -2957, 19399, 2155, 26016, + -2957, 19399, -2957, 52514, 57354, 63, 2094, 57354, -2957, -2957, + -2957, 19399, -2957, 4053, 4053, 4053, 19399, -2957, 19399, -2957, + -2957, -2957, 2293, 2225, -2957, 2225, 19399, 3461, 586, 3230, + 62179, 29, -2957, 44174, -2957, -2957, -2957, 52514, -2957, 45254, + -2957, 649, 253, 2110, 19399, 39173, 2342, -2957, -2957, 2373, + -2957, 2432, -2957, 2174, 497, 2191, -2957, -2957, -2957, -2957, + 1244, 634, 1643, 2011, 2039, -2957, 2119, 52514, 3461, 787, + 775, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, -2957, + -2957, -2957, -2957, -2957, 3461, 2555, 2348, 2563, 3461, 44174, + 1939, 19399, 124, -2957, 1137, 2558, -2957, -2957, 2628, 2225, + 2139, 22453, 2140, -2957, 2136, 62179, 44174, 2277, -2957, -2957, + 2147, -2957, -2957, 134, 19399, -2957, 39718, 2148, 2150, 2588, + 1797, 2155, 2155, -2957, -25, -2957, -2957, 2562, 31378, 2520, + 1112, 649, 2162, 1144, -2957, -2957, -2957, -2957, -2957, 2602, + -2957, 39213, 2389, 155, 2375, 2110, 19399, -2957, 2226, -2957, + -2957, -2957, 2620, -2957, -2957, 48642, 2158, 2039, 2011, 1830, + 2039, 2377, -2957, 2378, -2957, 2163, 39253, 62179, 62179, 1643, + 31378, 62179, 2161, 2155, -2957, 2164, -2957, -2957, -2957, 51546, + -2957, 2166, 2167, -2957, -2957, -2957, 19399, 147, -2957, -2957, + 2217, 52514, 1145, 79, 2373, 37188, -2957, 45254, 1688, 253, + 2472, -2957, -2957, -2957, -2957, 137, 2390, -2957, 2393, -2957, + 44174, -2957, 3461, 48642, -2957, -2957, -2957, -2957, -2957, -2957, + 31378, 2558, -2957, 324, -2957, 1514, -2957, 324, -2957, -2957, + -2957, -2957, 634, -2957, 1476, 22962, 22962, 22962, 2177, 3461, + -2957, 1514, -2957, 2300, 2375, -2957, -2957, -2957, -2957, -2957, + 175, 175, 2560, -2957, 2240, -2957, 2039, 1151, 62179, 1754, + -2957, 1754, 24489, 2327, 197, 43180, 2542, -2957, 2542, 2542, + -2957, -2957, -2957, 36219, -2957, -2957, 2664, -2957, 202, -2957, + -2957, -2957, 1643, 324, -2957, -2957, 2654, -2957, -2957, -2957, + -2957, -2957, 204, -2957, -2957, -2957, 1514, 649, -2957, -2957, + -2957, 1514, 1754, 23471, 2333, -2957, 2403, -2957, -2957, -2957, + -2957, -2957, -2957, -2957 }; /* YYPGOTO[NTERM-NUM]. */ static const yytype_int16 yypgoto[] = { - -2936, -2936, -2936, 2058, 92, -2936, -2936, 221, -2936, 1016, - -2936, 206, -629, 573, -2936, 102, 2939, 2556, 2459, 1050, - -497, -668, -1290, -924, 104, -1109, 7, -2936, -2936, -2936, - -2936, -2936, -391, 272, -2936, -2936, -527, -2483, -480, -2936, - -2848, -2931, -2936, -2936, -626, -2935, -1986, 106, -2936, -2936, - 107, 3, -2060, 113, 915, -2936, -2472, 118, -704, -1145, - -874, -1128, -2936, -41, 120, 1239, -2936, 8, -2100, -2785, - -496, -2936, -602, -2936, -262, -2936, -535, -2936, -790, -545, - -578, -2722, -1100, -2936, 1626, -299, -2936, 613, -2936, -2466, - -2936, -2936, 601, -2936, -1115, -2936, -2138, 191, -528, -2238, - -2484, -2091, -709, 265, -533, 243, -2056, -1058, -2936, 624, - -2936, -518, -2936, -686, -1872, 121, -2936, -2936, 1386, -765, - -2936, 1436, 122, 1470, -2053, 33, 51, -2936, -2936, -2936, - -2936, -617, 565, -1156, -2936, 487, -2936, -2936, -2936, -2936, - -107, 237, -2936, 2, 2783, -40, -29, -2936, -24, -2936, - -2936, -2936, 655, -2936, -2936, 18, 64, 1696, -2936, -981, - -2936, -1601, 811, -2936, 1838, 1850, -2129, -627, -60, -2936, - 700, -1627, -2047, -541, 1112, 1697, 1686, 440, -2208, -2936, - -473, -2936, 211, -2936, -2936, 690, 1153, -1518, -1516, -2936, - -2121, -2936, -392, -279, -2936, -2936, -2936, -2936, -2936, -2418, - -2735, -536, 1128, -2936, 1705, -2936, -2936, -2936, -2936, 139, - -1471, 2793, 730, 228, -2936, -2936, -2936, -2936, 180, -2936, - 881, -123, -2936, 2210, -524, -610, 1890, -584, 232, -1696, - -5, 2219, 495, -2936, -2936, 488, -2013, -1476, 448, -195, - 1477, -2936, -2936, -525, -1280, -781, -2936, -2936, 264, -462, - -2936, -2936, -2936, -162, 410, -2936, -2936, 1060, 1935, -2936, - -591, 2119, -482, -745, 1909, -860, 1913, -863, -849, -859, - 1917, 1918, -1472, 4522, -196, -658, -2147, -2936, 1002, 45, - -2936, -1432, 109, -2936, -2936, -2936, -2936, -2170, -2936, -366, - -2936, -355, -2936, -2936, -2936, -475, -2446, -2936, 3524, 859, - -2936, -2936, 460, -2936, -2936, -2936, -2936, -1485, -2936, 5360, - 760, -2936, -1947, -2936, -2936, -921, -655, -1062, -948, -1236, - -1294, -2936, -2936, -2936, -2936, -2936, -2936, -1299, -1738, -204, - 820, -2936, -2936, 901, -2936, -2936, -2936, -1676, -2061, -2936, - -2936, -2936, 825, 1392, 155, -606, -1578, -2936, 864, -2266, - -2936, -2936, 459, -2936, -474, -1075, -2348, 360, 16, -2936, - 3169, -2429, -2936, -2936, -551, -2586, -1080, -789, -2936, 123, - -2936, 416, 124, -1620, -2936, 10, -2936, -314, -2936, -2936, - -2473, -2936, 125, 128, 2279, -2936, -2936, -2936, -476, -2936, - -488, -483, -2936, -2936, 22, -900, 1474, -2936, 129, 621, - -2936, 933, -2936, 711, 131, 103, 1455, 132, 1264, -2936, - -2936, -2936, 53, -529, 417, -2936, 1257, -2936, -2936, 2872, - -2936, 133, -2936, 974, 134, -2936, -2936, 76, 2810, 135, - 15, -2835, 136, -2694, -1624, -7, -2936, -2936, -2936, -542, - -2936, -2432 + -2957, -2957, -2957, 2018, 92, -2957, -2957, 186, -2957, 984, + -2957, 187, -638, 537, -2957, 102, 3240, 2203, 3648, 150, + -497, -711, -1293, -918, 111, -1089, 12, -2957, -2957, -2957, + -2957, -2957, -433, 235, -2957, -2957, -571, -2471, -519, -2957, + -2804, -2931, -2957, -2957, -665, -2867, -1990, 112, -2957, -2957, + 114, 5, -2034, 115, 878, -2957, -2453, 118, -709, -1148, + -874, -1146, -2957, -77, 119, 1235, -2957, 10, -2088, -2793, + -532, -2957, -635, -2957, -298, -2957, -581, -2957, -852, -589, + -624, -2703, -1081, -2957, 1588, -343, -2957, 569, -2957, -2445, + -2957, -2957, 557, -2957, -1109, -2957, -2130, 146, -568, -2500, + -2467, -2073, -694, 225, -576, 201, -2043, -1176, -2957, 580, + -2957, -558, -2957, -686, -2334, 120, -2957, -2957, 1343, -777, + -2957, 1393, 121, 1427, -2027, 18, 21, -2957, -2957, -2957, + -2957, -590, 518, -1166, -2957, 441, -2957, -2957, -2957, -2957, + -153, 188, -2957, 7, 3386, -6, -28, -2957, -22, -2957, + -2957, -2957, 610, -2957, -2957, 16, 64, 1641, -2957, -973, + -2957, -1592, 645, -2957, 1790, 1791, -2127, -637, -63, -2957, + 650, -1608, -2065, -521, 1072, 1631, 1635, 400, -2822, -2957, + -513, -2957, 199, -2957, -2957, 647, 1116, -1497, -1490, -2957, + -2091, -2957, -432, -322, -2957, -2957, -2957, -2957, -2957, -2292, + -2289, -510, 1088, -2957, 1650, -2957, -2957, -2957, -2957, 25, + -1463, 2755, 687, -48, -2957, -2957, -2957, -2957, 130, -2957, + 841, -161, -2957, 2176, -533, -625, 1845, -42, 262, -1678, + 34, 2178, 448, -2957, -2957, 442, -2019, -1467, 402, -241, + 1431, -2957, -2957, -538, -1290, -787, -2957, -2957, -467, -525, + -2957, -2957, -2957, 1426, 1582, -2957, -2957, 1930, 2367, -2957, + -620, 2427, -482, -772, 1864, -867, 1872, -878, -855, -866, + 1874, 1876, -1473, 4769, 1370, 1134, -2139, -2957, 956, 45, + -2957, -1429, 109, -2957, -2957, -2957, -2957, -2614, -2957, -408, + -2957, -404, -2957, -2957, -2957, -1572, -2956, -1599, -2957, 3663, + 800, -2957, -2957, 412, -2957, -2957, -2957, -2957, -1472, -2957, + 5495, 701, -2957, -1941, -2957, -2957, -932, -648, -1044, -948, + -1230, -1288, -2957, -2957, -2957, -2957, -2957, -2957, -1486, -1726, + -262, 766, -2957, -2957, 854, -2957, -2957, -2957, -1686, -2021, + -2957, -2957, -2957, 776, 1345, 113, -598, -1559, -2957, 812, + -2275, -2957, -2957, 417, -2957, -471, -1091, -2336, 66, 135, + -2957, 3856, -2440, -2957, -2957, -557, -2560, -1092, -790, -2957, + 125, -2957, 374, 126, -1615, -2957, 13, -2957, -356, -2957, + -2957, -2482, -2957, 127, 128, 2236, -2957, -2957, -2957, -452, + -2957, -481, -475, -2957, -2957, 1, -895, 1436, -2957, 129, + 123, -2957, 885, -2957, 694, 131, 122, 1410, 132, 1209, + -2957, -2957, -2957, 35, -503, 371, -2957, 1220, -2957, -2957, + 2836, -2957, 136, -2957, 528, 141, -2957, -2957, 91, 2757, + 144, 32, -2828, 145, -2551, -1609, -7, -2957, -2957, -2957, + -541, -2957, -2427 }; /* YYTABLE[YYPACT[STATE-NUM]]. What to do in state STATE-NUM. If positive, shift that token. If negative, reduce the rule which number is the opposite. If zero, do what YYDEFACT says. If YYTABLE_NINF, syntax error. */ -#define YYTABLE_NINF -1968 +#define YYTABLE_NINF -1977 static const yytype_int16 yytable[] = { - 502, 716, 66, 56, 1169, 679, 825, 52, 61, 690, - 78, 1309, 1349, 972, 502, 96, 2102, 1100, 73, 1459, - 1421, 922, 1733, 500, 1744, 1889, 2069, 1699, 1631, 1946, - 806, 2465, 665, 64, 1515, 1705, 73, 500, 661, 1311, - 586, 1734, 1405, 1070, 694, 1985, 1415, 1709, 1262, 695, - 1798, 65, 499, 88, 1355, 2475, 1687, 2508, 2448, 1396, - 2450, 1749, 1399, 1738, 502, 502, 625, 1133, 1436, 1802, - 2203, 1434, 1438, 818, 833, 2547, 2548, 2549, 2864, 1168, - 916, 1174, 1437, 1178, 827, 2004, 2005, 500, 500, 1605, - 2429, 2866, 49, 809, 1608, 954, 702, 2501, 2842, 710, - 975, 2840, 50, 1820, 51, 2464, 54, 55, 677, 658, - 923, 2878, 2881, 57, 1545, 1543, 625, 625, 58, 949, - 59, 62, 63, 76, 77, 79, 2558, 633, 80, 85, - 627, 86, 87, 92, 93, 95, 97, 2073, 2733, 2669, - 2226, 2222, 2738, 2122, -491, 629, 996, 2125, 1122, -494, - 513, 3264, 1866, 1125, 2343, 1111, 3063, 1868, -1815, 1357, - 619, -1815, 629, 1357, 2561, 1335, 1336, 1086, -831, -1955, - -1955, 3049, -823, -1195, -1214, 2565, -1806, -1214, 2572, 1357, - 627, 627, -1211, -1211, -1946, -1946, -1806, 3067, 669, 3002, - -1823, 1142, 1347, 685, 1701, -831, 2019, 1143, -1964, -1964, - -1941, -1941, -828, -1815, 2019, 1736, -828, 1613, -1215, 629, - 619, 619, 2394, 996, -1823, -1215, -1212, -1212, -1966, -1966, - -790, -803, -818, 629, 629, 629, 682, 1357, 3140, 2372, - 2373, 2653, 1082, 1609, 1408, 629, 2135, -448, 1408, 996, - 2379, 2985, -491, 996, 2383, 3346, 2579, -494, 516, 2509, - 2614, 2616, 3153, 2619, 3036, 1701, 1690, -627, 1710, 516, - 995, 3224, 686, 1866, 1329, 743, 2536, 1867, 1868, 996, - -1057, -1968, -1968, -1968, 1429, 1795, 687, 3013, -1057, 1561, - 1453, 1262, 1718, 1355, 1563, 1721, 1722, 2968, 3064, 2970, - 3289, 3390, -214, 2882, 1710, 1367, 1368, -214, 1144, 691, - 692, 1330, 3370, 3065, 3383, 2056, 597, 21, 1089, 2219, - 517, 818, 3338, 2057, 1571, 683, 1747, 2366, 2367, 2368, - 2706, 517, 1443, 955, 2975, 1748, 3251, 2705, 2999, 2707, - 687, 809, 1647, 2355, 2152, 2209, 2353, 3324, 687, 3, - 4, 1561, 2756, 1610, 3268, 3232, 1573, 2636, 1693, 1118, - 2040, -1078, 2066, 1561, 1710, 98, 2510, 1562, 1563, -1078, - 106, 2976, 2747, 1373, 1374, 2305, 3168, 3061, 2757, 3332, - 2306, 2152, 532, 1634, 1717, 1486, 1571, 561, 922, 533, - 1539, 1493, 3353, 1728, 2440, 3218, 2356, 3219, 1571, 635, - 1866, 3058, 2153, 811, 1867, 1868, 3035, 1906, 3333, 2746, - 3057, 3000, 99, 676, 676, 3169, 3233, 1729, 1573, 975, - 1866, 3069, 1335, 1336, 1867, 1868, 1732, 3062, 1262, 3311, - 1573, 954, 3314, 3378, 1120, 2307, -491, 2729, 1347, 2498, - 1906, -494, -1940, -1940, 3014, 1422, 1310, 1648, 3250, 922, - 2501, 598, 2501, 774, 1358, 25, 26, 27, 1358, 3151, - 3391, 1145, 2504, 2883, 2666, 1711, 2888, 3384, 3170, 956, - 1959, 3270, 706, 1665, 1358, 1611, 1708, 3171, 2387, 2041, - 3354, 1090, 1453, 1091, 1531, 1331, 1739, 1377, 1378, 3225, - 1686, 1083, 688, 3339, 520, 2042, 521, 3085, 3066, 3172, - 2043, 1711, 1750, -491, 2210, 520, 2067, 521, -494, 3129, - 1837, 818, 2136, 1695, 3371, 2653, 1146, 3341, 32, 1145, - 1147, 524, 1358, 2874, 2208, 2905, 719, 3376, 3003, 1708, - 670, 809, 524, 679, 2357, 1682, 1683, 2354, 2044, 1635, - 1737, 2286, 3017, 3312, 1561, 1960, 1874, 2886, 1562, 1563, - 1148, 922, 922, 3141, 1551, 37, 1949, 684, 820, 2554, - 2555, 1711, 1985, 2020, 1084, 1708, 921, 2896, 2220, 3044, - 3173, 2431, 3313, 2316, 3182, 744, 3347, 990, 2511, 1571, - 2512, 714, 3183, 3174, 679, 715, 2147, 39, 2623, 1633, - 73, 930, 3160, 2505, 2625, 918, 1332, 3372, 42, 2726, - 2122, 2537, 1684, 2478, 2344, 514, 502, 1685, 2513, 1614, - 1359, 1573, 43, 2452, 1395, 2842, -491, 502, 2840, 2566, - 1149, -494, 2575, 1785, 3373, 1708, 2473, 1696, 1333, 500, - 1398, 1626, 2776, 696, 2467, -1057, 44, 1350, -763, 1723, - 500, 811, 596, 694, 1134, 3299, 2274, 1883, 695, 3132, - 45, 2993, 502, 502, 3133, -491, 2016, -491, 625, 1351, - -494, 1734, -494, -1815, 1350, 2727, -1815, 2558, 2072, 625, - 2058, 1528, 1087, 1352, 1528, 502, 714, -823, -1195, -1214, - 715, -1806, -1214, 818, 1751, 3106, 1351, 2045, 2647, 1087, - 2292, -1806, 1539, 66, 56, -1823, 3249, 1526, 52, 61, - 1354, 78, 1093, 809, 2777, 3256, 96, -828, -1815, 73, - 3255, 927, 3087, -1215, 1556, 943, -1078, 502, 1259, -1823, - -1215, 1305, 627, 2208, 64, 1337, 1341, -818, 1606, 2074, - 2934, 1412, 1413, 627, 647, 1412, 1413, 658, 658, 2128, - 658, 3042, 65, 2626, 88, 3184, 2984, 1101, 2097, -1968, - -1968, -1968, 1452, 1878, 1879, 1880, 1881, 1882, 1883, 2084, - 1728, 3026, 3027, 945, 1799, 1800, 1118, 1092, 3135, -763, - 2637, 2638, 2639, 2640, 1594, 628, 3175, 3300, 2864, 3176, - 1724, 2441, 2085, 49, 1729, 2714, 2778, 3145, 1725, -214, - -214, 2986, 2168, 50, 2172, 51, 820, 54, 55, 1775, - 1561, 1453, 1453, 3322, 57, 1373, 1374, 1453, 3120, 58, - 3122, 59, 62, 63, 76, 77, 79, 3047, 1898, 80, - 85, 2324, 86, 87, 92, 93, 95, 97, 806, 2098, - 2421, 811, 648, 3011, 1305, -647, 3058, 1112, 2715, 1498, - 2351, 1120, 818, 1350, 1809, 1591, 1592, 1593, 1594, 2255, - 2129, 825, 946, 981, 1822, 1142, 2182, 2422, 3301, 666, - 1826, 1143, 809, 2750, 982, 1351, 2185, 1573, 1710, 2188, - 2842, 818, 818, 2840, 672, 922, 1958, 1539, 2855, 2351, - 2856, 1913, 1880, 1881, 1882, 1883, 1988, 818, 1121, 1499, - 1142, 809, 809, 1262, 2001, 1262, 1143, 681, 1934, 738, - 1878, 1879, 1880, 1881, 1882, 1883, 3012, 809, 3138, 2002, - 1650, 1710, 1406, 3019, 3392, 1407, 1889, 1350, 680, 1377, - 1378, 3024, 3381, 921, 2404, 3196, 1985, 3125, 1830, 830, - 697, 502, 502, 1823, 916, 1113, 1522, 502, 45, 1351, - 502, 502, 2360, 502, 502, 502, 502, 2298, 516, 2751, - 1650, 73, 2676, 1352, 1651, 1710, 1904, 1710, 3398, 502, - 2325, 629, 1144, 2279, 1551, 2278, 502, 2281, 2283, 2284, - 2326, 1483, -1258, 2295, 2423, 812, 1350, 2280, 2282, 2424, - 2512, 2550, 500, 718, 921, 1305, 820, 1832, 1701, 500, - 1742, 958, 996, 2597, 1651, 725, 959, 1144, 1351, 1702, - 2601, 2111, 2603, 811, 739, 502, 679, 2453, 2789, 2454, - 517, 625, 1354, 2752, 1834, 2753, 1345, 1346, 625, 996, - 3285, 3286, 2181, 502, 1589, 1590, 1591, 1592, 1593, 1594, - 2959, 736, 2022, 2535, 502, 502, 502, 1339, 502, 502, - 741, 1259, 2262, 676, 1142, 3228, 2006, 2760, 731, 1405, - 1143, 1644, 2476, 960, 740, 2473, 714, 996, 1652, 1348, - 715, 2530, 2230, 1734, 2285, 1711, 2484, 1653, 2287, 502, - 2144, 2289, 3326, 1118, 2531, 627, 729, 2539, 1119, 2543, - -1937, -1937, 627, 631, 631, 732, 921, 921, 2763, 818, - 2538, 3278, 2231, 1391, 742, 1638, 1639, 2180, 1645, 1701, - 695, 695, 961, 695, 745, 619, 2545, 1653, 1711, 809, - 1704, 3152, 619, 1087, 101, 1145, 2191, 2546, 1821, 502, - 2568, 2198, 961, 502, 502, 2425, -1938, -1938, 1654, 1708, - 2204, 2205, 2206, 502, 502, 502, 2426, 962, 502, -571, - 1907, 746, 714, 2685, -571, 1908, 1550, 730, 1120, 2259, - 1145, 1144, 1711, 1484, 1711, 2580, 1543, 962, 820, 21, - 747, 2588, 811, 964, 2901, 1142, 831, 713, 1654, 734, - 1172, 1143, 1708, 963, 1147, 1476, 1735, 729, 3114, 930, - 1762, 3245, 2156, 964, 520, 2183, 521, 939, 1821, 3115, - 2186, 811, 811, 1502, 1479, 1121, 714, 1506, -1195, 2060, - 715, 1490, 948, 2061, 1173, 1176, 1491, 811, 523, 1147, - 1496, 524, 928, 812, -571, 965, 1708, 1763, 1708, 1453, - 1453, 1453, 1453, 1453, 1453, 818, 929, 1453, 1453, 1453, - 1453, 1453, 1453, 1453, 1453, 1453, 1453, 2304, 931, 1177, - 731, 2308, 3143, 1624, 2310, 809, 1625, 2767, 730, 2168, - 2168, 2168, 3053, 1985, 1827, 108, 2349, 1828, 512, 966, - 721, -571, 722, 1864, 1865, 560, 967, 1305, 1897, 1885, - 1899, 1900, 1144, 2245, 1149, 2246, 1305, 732, 102, 3388, - 646, 1339, 1077, 1078, 659, 1080, 968, 103, 1948, -1939, - -1939, 1949, 2269, 723, 2768, 724, 1348, 25, 26, 27, - 1305, 1765, 934, 1485, 1145, 922, 968, -1942, -1942, 1149, - 2769, 2648, 733, 1950, 937, 2655, 1951, 820, 3379, 3356, - 3380, 2216, 104, 969, 2217, 2263, 1769, 2270, 1951, 2272, - 2271, 1770, 2271, 1772, 3367, 818, -1943, -1943, 1773, 2311, - 502, 1762, 2299, 2900, 938, 2300, 820, 820, 977, 2337, - 2385, 2317, 2338, 2386, 2388, 809, 3397, 2386, 940, 2143, - 32, 734, 820, 1147, 2526, 980, 2532, 2527, 2361, 2533, - 2362, 105, 1453, 1453, -1944, -1944, 2591, 2780, 1763, 1949, - 2419, 2345, 818, 2523, 3395, 2525, 1837, 2660, 2350, 3396, - 2386, 970, 2744, 1148, 818, 2787, 1259, 37, 2721, 941, - 2949, 2722, 809, 812, 2609, 2520, 942, 2522, 2770, 811, - 921, -1945, -1945, 1764, 809, 922, 2873, 2771, 2363, 2217, - 2364, 2779, 2861, 2788, 818, 1145, 2384, 1259, 943, 39, - 2384, 818, 2894, 502, 73, 2895, 2390, 989, 2902, 918, - 42, 2903, 2950, 3080, 809, 1951, 1951, 3214, 2743, 3215, - 2745, 809, 502, 502, 3081, 2449, 3109, 2386, 502, 1951, - 502, 3116, 1765, 1149, 1949, 502, 502, 502, 502, 818, - 990, 3126, 676, 818, 3127, 3159, 983, 1483, 2217, 818, - 502, 502, 988, 500, 1147, 502, 1453, 502, 3193, 809, - 502, 1951, 45, 809, 1543, 502, 991, 502, 502, 809, - 502, 1096, 3266, 3293, 502, 3267, 3294, 1115, 500, 1117, - 500, 1081, 625, 500, 1148, 1085, 1155, 3337, 500, 1102, - 3267, 500, 1103, 500, 502, 502, 1105, 500, 2564, 502, - 1955, 502, 2599, 2600, 502, 1952, 1156, 625, 3377, 625, - 2487, 3267, 625, -1947, -1947, 811, 73, 625, -1948, -1948, - 625, 1106, 625, 1259, 1108, 2069, 625, 502, 1109, 502, - 1116, 1953, 2473, 1135, 820, 502, 502, 502, 502, 502, - 502, 502, 502, 1259, 1137, 812, 627, 502, 502, 1954, - 1157, 1957, 502, 1139, 1149, 1140, 502, -1949, -1949, 502, - 502, 502, 502, 502, 502, 502, 502, 502, -1950, -1950, - 502, 627, 1141, 627, -1951, -1951, 627, 502, 961, 1305, - 1153, 627, 1154, 2168, 627, 3079, 627, 2843, 1171, 2172, - 627, 985, 986, 987, 1179, 814, 1175, 2104, -1952, -1952, - 502, 619, 695, 619, -1953, -1953, 619, 2994, 21, -1954, - -1954, 619, 1307, 962, 619, 1308, 619, -1956, -1956, 1312, - 619, 1314, 502, 1315, 3192, 811, 925, 1093, 631, 1492, - -1957, -1957, 1325, 502, 502, 1327, 2835, -1958, -1958, 964, - 1328, 2787, 1158, -1959, -1959, -1960, -1960, -1961, -1961, -1963, - -1963, 2318, 2319, 2320, 2321, 2322, 2323, 1333, 979, 2327, - 2328, 2329, 2330, 2331, 2332, 2333, 2334, 2335, 2336, 1340, - 820, 1337, 811, 3355, -1965, -1965, 1341, 3357, 1343, 1259, - 1344, 1259, 1356, 1079, 811, 3043, -1967, -1967, 1753, 1754, - 1925, 1159, 1926, 2156, 1361, 1928, 2123, 2124, 1362, 1160, - 1932, -622, -622, 1935, 812, 1936, 1375, 1376, 1364, 1940, - 1392, 1161, 1142, 1393, 811, 1377, 1378, 2606, 1143, 502, - 1397, 811, 1400, 1305, 2990, 2991, 502, 502, 1423, 2961, - 1424, 3393, 922, 812, 812, 1401, 818, 1427, 1824, 1470, - 1825, 1473, 1162, 2028, -626, -626, 25, 26, 27, 812, - -625, -625, 1430, 1453, 1453, 1457, 809, 1472, 1487, 811, - 1305, 1480, 968, 811, 1488, 2249, 1494, 502, 1495, 811, - 2029, 1501, 1505, 3192, 1618, 1620, 1518, 2001, 995, 1485, - 820, 1866, 1507, 1519, 1521, 1867, 1868, 2473, -794, 1869, - 1870, 1871, -801, 1164, 1259, 1526, 45, -647, -648, -503, - -791, 1305, 502, -792, 2416, 2417, 1530, -795, 1142, 32, - 921, 1531, -793, -503, 1143, 3192, 1165, 995, -503, 1144, - 1866, 2893, 1535, 1544, 1867, 1868, 1554, 820, 1869, 1870, - 1871, 502, 502, 814, 502, 1167, 1600, 1604, 1637, 820, - 502, 502, 502, 502, 502, 502, 37, 1616, 502, 502, - 502, 502, 502, 502, 502, 502, 502, 502, 2884, 1602, - 1627, 2868, 2737, 502, 502, 3192, 1628, 502, -503, 820, - 516, 1632, 1636, 1119, 502, 1672, 820, 1121, 39, 1674, - 3157, 1676, 2030, 1688, 1706, 1707, 3022, 2031, -503, 42, - 685, 1708, 2032, 1714, -1258, 1713, 502, 1715, 1719, 1726, - 502, 1727, 502, 2860, -572, 1731, 502, 1741, 1746, -572, - 1755, 1759, 1767, 1768, 820, 1144, 1777, 1780, 820, 1781, - 921, 1783, 1305, 500, 820, 818, 1787, 1784, 1793, 500, - 1786, 1794, 517, 502, 1801, -503, 996, 1803, 1804, 1821, - 1811, 45, 10, 1812, -503, 809, 1453, 3343, 1815, 1816, - 1414, 812, 625, 1817, 1818, 1829, 1855, 1857, 625, 686, - 1858, 1432, 1860, 1872, 502, 679, 3102, 1863, 1886, 502, - 502, 2927, 1145, 1543, 1894, 2609, 1895, 1902, 1905, -572, - 1923, 3247, 1927, 1937, 518, 2838, 1873, 1933, 1938, 1939, - 1944, 1561, 1947, 502, 502, 2945, 1994, 1991, 502, 1960, - 1996, 1999, 2137, 2018, 2025, 3208, 1998, 3031, 3032, 1997, - 1509, 1414, 502, 21, 2036, 502, 627, 502, 2037, 2039, - 2063, 2064, 627, 814, 1349, 1873, -572, 2189, 818, 2070, - 2082, 1147, 2148, 502, 1259, 687, 2083, 2087, 2033, 686, - 502, 2113, 2157, 502, 2160, 2112, 619, 2171, 809, 2034, - 2100, 2138, 619, 2175, 1874, 2177, 811, 2892, 519, 2103, - 502, 2190, 2114, 2115, 2116, 2139, 500, 2184, 1145, 2117, - 2142, 2134, 2187, 502, 2149, 1555, 2192, 2193, 2194, 2195, - 2145, 2199, 2200, 500, 1557, 1453, 2939, 812, -503, 2151, - 502, 502, 2150, 1874, 687, 625, 2214, 2218, 2223, 2232, - 818, 2234, 1607, 2235, 2236, 2250, 520, 502, 521, 502, - 2251, 1612, 625, 1414, 1414, 2253, 2256, 2257, 502, 2258, - 809, 2260, 1840, 2196, 522, 818, 2261, 1147, 2265, 2264, - 523, 1149, 2301, 524, 2266, 2374, 1259, 1259, 1259, 2267, - 2268, 2377, 2276, 2835, 2277, 809, 2288, 1734, 2309, 1891, - 1866, 25, 26, 27, 714, 2339, 1890, 2197, 715, 627, - 2370, 2378, 2420, 2346, 2400, 2347, 2348, 1841, 2392, 2352, - 2358, 2359, 679, 2391, 502, 2402, 627, 1365, 1366, 688, - 1305, 2376, 818, 2399, 2401, 2403, 1842, 2415, 2413, 619, - 3008, 2427, 3229, 2436, 2437, 814, 2442, 812, 2443, 2444, - 2456, 2460, 809, 2455, 1843, 1769, 619, 2458, 1844, 2461, - 1770, 820, 1772, 2468, 32, 2689, 2690, 1773, 2469, 2466, - 1453, -628, 2479, 2480, 2482, 2483, 2486, 1149, 2490, 502, - 1845, 2491, 2492, 1846, 2496, 2494, 2497, 2500, 2519, 2521, - 2534, 2553, 2540, 2551, 812, 811, 3030, 2563, 1847, 2541, - 2542, 37, 2552, 1367, 1368, 2562, 812, 1875, 1876, 1877, - 2567, 1878, 1879, 1880, 1881, 1882, 1883, 921, 2577, 2578, - 38, 2581, 2582, 1259, 2585, 2589, 2594, 2596, 2613, 2604, - 2605, 2624, 2644, 39, 2645, 2627, 812, 3146, 2630, 818, - 2621, 2631, 2632, 812, 42, 2633, 1875, 1876, 1877, 2656, - 1878, 1879, 1880, 1881, 1882, 1883, 2657, 2663, 43, 809, - 2673, 502, 2667, 2683, 2681, 2696, 2702, 1369, 1370, 1371, - 1372, 1373, 1374, 818, 2687, 1375, 1376, 2730, 2684, 2697, - 2711, 812, 44, 922, 500, 812, 2704, 2708, 2728, 1848, - 2734, 812, 3331, 809, 2732, 679, 45, 1849, 811, 2742, - 3229, 502, 2748, 2749, 814, 2755, 2775, 2791, 2857, 2858, - 2859, 2863, 2699, 625, 2867, 2876, 2871, 502, 1850, 2875, - 2872, 502, 502, 2880, 2654, 2889, 2891, 679, 502, 2898, - 2217, 2904, 2899, 814, 814, 2924, 2926, 2930, 502, 2935, - 2941, 502, 2942, 502, 1851, 2962, 2974, 818, 2835, 814, - 820, 502, 2969, 2972, 502, 502, 2981, 2983, 2897, 502, - 502, 3197, 2987, 3199, 2988, 2989, 502, 809, 3206, 3001, - 811, 2995, 1824, 2996, 2997, 3005, 3006, 627, 2736, 3007, - 3015, 502, 3018, 3020, 3021, 1377, 1378, 2001, 1955, -1211, - 3037, 502, 3025, 1952, 2720, 811, -1936, -1937, -1938, -1939, - 3038, 531, -1940, 1836, 73, 3210, 531, 1509, 1155, 3039, - -1941, 502, -1942, -1943, 3050, -1944, -1945, -1947, 531, 1953, - 815, 3209, 3207, -1948, 3212, 3040, -1949, -1950, 1156, -1951, - 3054, -1952, 531, 531, 3041, -1953, 1431, 1954, 629, 1957, - -1954, -1956, 2838, 3068, -1957, -1958, 3072, 3048, 3291, -1959, - 1259, 3055, 811, 2841, 1259, -1960, 1259, -1961, 3070, -1962, - -1963, -1964, 3295, -1965, -1966, -1967, -1212, 502, 3073, 502, - 1969, 3076, 1157, 3082, 3083, 3086, 3088, 1970, 1971, 3094, - 3097, 1972, 1973, 1974, 3090, 818, 3096, 3100, 3101, 3123, - 3103, 531, 3108, 498, 509, 2865, 3128, 3010, 530, 1769, - 1379, 1380, 3124, 530, 1770, 809, 1772, 616, 3131, 630, - 630, 1773, 2760, 3136, 634, 530, 642, 3134, 2761, 642, - 3284, 2885, 660, 664, 2887, 820, 664, 3148, 2716, 530, - 530, 2762, 3149, 3156, 3158, 3279, 3164, 3165, 2584, 679, - 1365, 1366, 3181, 3201, 3166, 3179, 3180, 502, 3194, 3202, - 820, 814, 3195, 2763, 3216, 2764, 3198, 616, 616, 811, - 3211, 3236, 3014, 3204, 1158, 3240, 3242, 502, 502, 3243, - 3246, 3252, 502, 3258, 3259, 502, 3260, 3264, 812, 3269, - 660, 3271, 21, 3273, 502, 2965, 3276, 664, 530, 664, - 664, 664, 1560, 811, 2075, 1561, 502, 3281, 2654, 1562, - 1563, 502, 502, 3277, 817, 3282, 502, 820, 679, 3283, - 3288, 502, 3290, 1159, 502, 502, 1367, 1368, 3292, 502, - 1305, 1160, 716, 502, 3297, 3302, 3308, 502, 3309, 3315, - 1571, 3310, 3078, 1161, 502, -503, 3316, -1968, 3317, 3335, - 3325, 2765, 3327, 3319, 3345, 3348, 3329, 3330, 3350, -503, - 500, 3365, 3368, 3346, -503, 3347, 3382, 3389, 815, 3394, - 2176, 2517, 1573, 2794, 1162, 2790, 3223, 811, 2723, 1107, - 3323, 3287, 3387, 2290, 3028, 3375, 3306, 3144, 502, 625, - 1369, 1370, 1371, 1372, 1373, 1374, 502, 814, 1375, 1376, - 3351, 3344, 3369, 3178, 2499, 2524, 3342, 2838, 1698, 2879, - 3349, 2759, 2495, 3340, -503, 502, 2844, 2766, 1819, 1797, - 2717, 1766, 2767, 67, 820, 1164, 2559, 2610, 3077, 2485, - 25, 26, 27, 1477, -503, 2418, 2862, 2141, 1629, 2635, - 2471, 67, 2109, 3328, 618, 1478, 3272, 2481, 1165, 1668, - 818, 3200, 2140, 627, 620, 3105, 2472, 2457, 2841, 67, - 2382, 1669, 933, 1444, 947, 2628, -1968, 1167, 678, 2768, - 809, 1673, 2933, 2629, 2659, 3150, 3275, 812, 1810, 2297, - 1433, -503, -1968, 619, 1435, 2769, 3274, -1968, 1439, 1440, - -503, 2739, 2740, 32, 618, 618, 693, 2411, 3364, 2665, - 502, 2474, 2395, 2434, 2412, 2432, 2973, 814, 1377, 1378, - 1914, 2675, 2709, 3262, 679, 811, 926, 67, 3060, 2792, - 2389, 1831, 1853, 2080, 2724, -1968, 534, 1414, 921, 2078, - 37, 502, 820, 2303, 2845, 2846, 2847, 2848, 2849, 2850, - 2851, 2852, 2853, 2854, 703, 0, 0, 502, 0, 0, - 0, 0, 817, 0, 814, 0, 0, 0, 815, 0, - 0, 0, 39, 0, 0, 0, 814, 1305, 0, 0, - 0, 0, 1479, 42, 502, 0, 0, 0, 1582, 0, - 812, 0, 0, 2770, 0, 502, 502, 43, 0, 502, - 0, 502, 2771, 0, 0, 0, 814, 0, 0, 0, - 0, 0, 0, 814, 0, 0, 0, 0, 0, 0, - 0, 44, 0, 0, 0, 0, 502, 0, 0, 668, - 0, 0, 0, 1379, 1380, 2718, 0, 0, 531, 0, - 0, 0, 0, 0, -503, 0, 0, 1414, 0, 0, - 502, 814, 0, 0, 0, 814, 0, 0, 0, 0, - 0, 814, 812, 0, 0, 0, 0, 531, 531, 1975, - 1976, 1977, 0, 1978, 1979, 1980, 1981, 1982, 1983, 0, - 2841, 0, 0, 0, -1968, 0, 0, 812, 2248, 0, - 705, 0, 708, 0, 712, 0, 0, 0, 0, 0, - 0, 0, 3161, 3162, 0, 0, 0, 0, 21, 0, - 714, 0, 0, 1105, 715, 502, 0, 0, 0, 502, - 0, 664, 0, 0, 0, 664, 0, 0, 664, 502, - 0, 502, 0, 502, 0, 530, 0, 502, 0, 502, - 815, 502, 500, 0, 812, 0, 0, 0, 0, 0, - 0, 502, 817, 0, 0, 0, 502, 0, 502, 0, - 0, 0, 0, 823, 530, 530, 502, 0, 0, 0, - 0, 625, 3009, 0, 0, 0, 3220, 0, 3222, 1259, - 811, 0, 531, 0, 502, 0, 0, 0, -1968, 0, - 0, 0, 0, 0, 0, 1589, 1590, 1591, 1592, 1593, - 1594, 0, 924, 509, 630, 0, 0, 0, 0, 498, - 0, 664, 0, 0, 0, 0, 0, 3254, 1155, 0, - 616, 0, 0, 0, 951, 951, 0, 0, 0, 951, - 974, 502, 3059, 3257, 0, 627, 0, 3261, 1156, 0, - 0, 502, 0, 642, 642, 642, 25, 26, 27, 0, - 0, 812, 21, 0, 502, 0, 0, 642, 642, 0, - 642, 0, 0, 0, 0, 0, 0, 0, 502, 0, - 2528, 0, 0, 664, 0, 0, 0, 0, 0, 530, - 0, 0, 1157, 0, 0, 812, 502, 0, 0, 0, - 664, 500, 0, 0, 0, 0, 0, 0, 0, 815, - 0, 0, 0, 0, 664, 0, 0, 0, 0, 32, - 502, -1701, 0, 0, 0, 2556, 0, 0, 0, 0, - 625, 0, 0, 0, 817, 0, 502, 0, 815, 815, - 0, 0, 2569, 500, 0, 2841, 0, 1259, 664, 664, - 664, 664, 0, 0, 815, 0, 37, 67, 814, 2783, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 812, - 502, 3352, 625, 958, 0, 0, 0, 0, 959, 0, - 0, 0, 0, 0, 1158, 502, 502, 0, 39, 0, - 0, 0, 0, 500, 627, 0, 0, 3366, 0, 42, - 25, 26, 27, 0, 1414, 67, 678, 0, 0, 0, - 0, 502, 2611, 43, 2612, 0, 0, 0, 2617, -1701, - 2620, 0, 625, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1159, 0, 960, 627, 44, 3147, 0, - 0, 1160, 0, 0, 0, 0, 932, 502, 0, 0, - 0, 45, 0, 1161, 0, 0, 0, 944, 3154, 3155, - 0, 0, 0, 32, -1701, 0, 0, 0, 0, 1471, - 0, 0, 0, 0, 0, 0, 0, 0, -1701, 3167, - 0, 0, 0, -1701, 1162, 0, 627, 0, -1701, 0, - 0, 0, 1504, 817, 0, 0, 0, -1701, 0, 0, - 37, -1701, 0, 0, 961, 693, 0, 812, 0, 0, - 0, 0, 1129, 0, 720, 0, 0, 0, 727, 0, - 0, 728, 817, 817, 67, 0, 0, 0, 0, 0, - 0, 0, 39, -1701, 0, 1164, 0, 0, 817, 962, - 0, 0, 0, 42, 0, 0, 0, 814, 951, 974, - 0, 664, 0, -1701, 1451, 963, 815, 43, 1165, 1546, - 951, 951, 958, 0, 0, 964, 530, 959, 0, 0, - 0, 0, 616, 0, 0, 1560, 0, 1167, 1561, 616, - 0, 44, 1562, 1563, 0, 0, -1968, -1968, -1968, 530, - 0, 0, 0, 0, 0, 45, 1508, 965, 0, 0, - -1701, 0, 0, -1701, 936, 0, 1520, 0, 0, -1701, - 0, 0, 0, 1571, 0, 0, 0, 0, 1130, 0, - 1572, 531, 531, 0, 960, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2834, 966, 0, 0, 0, 1573, 0, -1701, 967, 0, - 814, 1560, 0, 0, 1561, 0, 530, 0, 1562, 1563, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 2556, - -1701, 0, 0, 0, 0, 0, 0, 0, 1560, 0, - 0, 1561, 0, 0, 0, 1562, 1563, 0, 968, 1571, - 0, 0, 815, 961, 0, 0, -1968, 1136, 0, 995, - 0, 1621, 1866, 1623, 0, 969, 1867, 1868, 3336, 0, - 1869, 1870, 1871, 0, 0, 0, 1571, 0, 530, 530, - 0, 1573, 814, -1968, 664, 0, 666, 2678, 962, 0, - 0, 1317, 1319, 1322, 1324, 0, 0, 0, 0, 1574, - 817, 0, 0, 0, 963, 0, 1451, 814, 1573, 0, - 0, 0, 0, -1701, 964, 1575, 0, 664, 1697, 0, - 1576, 0, 812, -1701, 1607, 0, 67, 0, 0, 0, - 664, 0, 0, 970, 0, 0, 0, 0, 1131, 0, - -1701, 0, -1701, -1701, 2428, 0, 965, 664, 0, 0, - 0, 664, 0, 0, 0, 1743, 0, 0, 1579, 618, - 0, 0, 815, 0, 814, 0, 618, 1929, 0, 0, - 0, 2688, 0, 0, 0, -1968, 0, 0, 2971, -1701, - 0, 0, -1701, -1701, -1701, 0, 0, 0, 0, 0, - 966, -1968, 0, 0, 0, 0, -1968, 967, 0, 0, - 0, 1523, -1968, 0, 0, 0, 0, 0, 0, 815, - 0, 1582, 1527, 0, 0, 0, 0, 0, -1968, 0, - 0, 815, 0, -1968, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, -1968, 0, 817, 968, 0, 1541, + 505, 975, 1172, 828, 719, 56, 693, 66, 1265, 1352, + 61, 1462, 52, 78, 505, 1634, 73, 1747, 64, 589, + 809, 65, 1736, 694, 695, 925, 1737, 1103, 1424, 1312, + 1358, 1892, 96, 812, 73, 88, 1702, 1518, 2105, 682, + 2478, 1949, 1314, 664, 1408, 697, 622, 1690, 2072, 1418, + 1399, 698, 502, 1402, 1073, 2451, 1439, 2453, 1708, 1801, + 1136, 1988, 1805, 2206, 505, 505, 628, 1437, 1441, 1712, + 1752, 668, 2511, 1171, 2468, 1177, 821, 1181, 535, 1440, + 2550, 2551, 2552, 564, 836, 2869, 1741, 2007, 2008, 957, + 830, 919, 49, 2843, 978, 638, 622, 622, 2867, 2432, + 1608, 705, 50, 1823, 2467, 1611, 2845, 1546, 680, 679, + 679, 51, 54, 2504, 55, 57, 628, 628, 58, 59, + 62, 63, 952, 1548, 661, 76, 77, 79, 80, 85, + 630, 86, 87, 2672, 2884, 2881, 92, 926, 713, 1338, + 1339, 93, 503, -1824, 95, 97, 636, 2561, 2076, 2225, + 1564, -1964, -1964, 2736, -491, 1566, 503, 2741, 2229, 1869, + 999, 516, -1950, -1950, 1871, -1824, 1350, 2125, 709, 2346, + 1089, 1125, -823, 634, 634, 632, 2128, -1220, 3066, -494, + 630, 630, 1128, -1220, 1114, 1574, -1201, -1975, -1975, 1693, + -1815, -1217, -1217, 2564, 2575, -831, 1360, -828, -1955, -1955, + -1832, -828, 3267, -831, 2568, -1221, 503, 503, -1973, -1973, + 1616, 632, 3005, -1218, -1218, 999, 1411, 1576, 2022, 672, + -1815, 999, -1824, -1063, 1085, 2022, 685, -1832, 1360, -1221, + -503, -1063, -790, -803, -818, 632, 632, 2059, 3375, 632, + -1084, 1265, 2656, 688, -503, 2060, 1360, 3156, -1084, -503, + 2375, 2376, -491, 999, 2763, 3350, 2988, 716, 3143, 1358, + 2138, 2382, 1612, 1411, 632, 2386, 2617, 2619, 999, 2622, + -627, 812, 1332, 3020, 746, 2397, 1360, -494, -448, 998, + 2031, 1696, 1869, 3039, 1432, 1721, 1870, 1871, 1724, 1725, + 1872, 1873, 1874, 690, 1713, 2766, 1798, 1564, 1713, 1342, + -503, 1565, 1566, 1092, 2582, 958, 3404, 2032, 1713, 1333, + 1446, 1713, 3227, 689, 821, 3399, 686, 3052, 600, 1750, + -503, 1351, 2212, 3302, 2779, 2709, 3335, 1704, 1751, 2369, + 2370, 2371, 1574, 3070, 3, 4, 3254, 2885, 1739, 3067, + 3327, 1713, 2358, 2222, 2639, 1869, 2327, 2356, 3342, 1870, + 1871, 1637, 690, 2507, 3068, 1394, 2069, 3123, 3388, 3125, + 2708, 3292, 2710, -763, 1576, 2100, 1564, -503, 2729, 3016, + 2628, 1720, 1613, 2750, 1704, 98, -503, 2443, 1265, 2319, + 3405, 2978, 925, 1731, 1909, 2539, 1338, 1339, 3038, 690, + 1542, 3271, 669, -214, 3060, 2359, 1121, 2780, -214, 3288, + 3289, 1574, 1350, 978, 1564, 3072, 3386, 1732, 1565, 1566, + 2347, 3061, 2732, 2087, 2749, 957, 2971, 1909, 2973, 2979, + 2033, 1735, 99, -1949, -1949, 2034, 3002, 3383, 106, 3314, + 2035, 1742, 3317, 1576, 1745, 2730, 2088, -491, 1698, 1574, + 3185, 3376, 959, 925, 3022, 1425, 2101, 3411, 3186, 536, + 3146, 3329, 3027, 3357, 601, 2770, 2504, 3303, 2504, 3090, + 3253, 812, -494, 2669, 2538, 1875, 1093, 2891, 1094, 599, + 3406, 1576, 1123, 777, 3199, 3400, 1086, 3336, 3154, 2781, + 2390, 1962, 2213, 1361, 2508, 1334, 2328, 1869, 1876, 1534, + 1689, 1870, 1871, 1714, -763, 1614, 2329, 1714, 2886, 3138, + 3003, 2070, 3345, 2771, 821, -491, 3337, 1714, 1638, 1840, + 1714, 1753, 3171, 3389, 3132, 1361, 2656, 3358, 3148, 2772, + 2629, 3343, -503, 2211, 722, 3377, 1711, 3017, 3381, 2139, + -494, 2289, 3228, 1361, 2360, 691, 1685, 1686, 3304, 2357, + 1714, 3069, 2877, 1342, 3006, 925, 925, 1554, 2908, 1087, + 687, 823, 3378, 1699, 673, 3315, 1877, 1711, 1351, 924, + 682, 1711, 993, 1361, 2557, 2558, 2889, 921, 1988, 949, + 1952, 1711, 933, 2899, 1711, 3144, 747, 2023, 2036, 3351, + -1063, 73, 1835, -647, 2434, 2150, 631, 999, 717, 2037, + 3014, 1108, 718, 2626, 2061, 2223, 1335, -1084, 741, 505, + 2843, 1963, 1617, 3047, 1711, 2455, 3163, 675, 1636, 517, + 505, 682, 3187, 2845, 2125, 1687, 3088, 2569, 2773, -491, + 3407, 1336, 2481, 1348, 1349, 717, 1688, 2774, 833, 718, + 2578, 1788, 2476, 812, 1137, 2470, 1737, 697, 1597, 632, + 1362, -1824, 948, 698, -494, 505, 505, 1886, 1629, 1531, + 3235, 628, 1529, 2277, 3135, 2019, 2996, 1353, -491, 3136, + -491, 2075, 628, -1824, 3015, 1353, 1740, 1754, 505, 3109, + -823, 1531, 1398, 2295, 1096, -1220, 821, 946, 2561, 1354, + 3252, -1220, 717, -494, -1201, -494, 718, 1354, -1815, 56, + 1401, 66, 1542, 1355, 61, -828, 52, 78, -1832, 2650, + 73, 1357, 64, -1221, 1090, 65, 1415, 1416, 3316, 1559, + 505, 1262, 3236, 2540, 1308, 630, 96, 3259, -1815, 88, + -1824, 1090, 2211, 930, 3258, -1832, 630, -1221, 1802, 1803, + 1340, 1344, -818, 1609, 2077, 2987, 3045, 2937, 3029, 3030, + 1104, 503, 679, 2155, 632, 661, 661, 1650, 661, 928, + 1121, 634, 503, 1415, 1416, 1095, 1376, 1377, 1647, 650, + 1115, 1878, 1879, 1880, 2753, 1881, 1882, 1883, 1884, 1885, + 1886, 2444, 2640, 2641, 2642, 2643, 49, 1080, 1081, 1778, + 1083, 982, 1594, 1595, 1596, 1597, 50, 3325, 2867, 823, + 2989, 2171, 812, 2175, 2155, 51, 54, 2512, 55, 57, + 1564, 2156, 58, 59, 62, 63, 1082, 2131, 809, 76, + 77, 79, 80, 85, 2308, 86, 87, 2354, 1901, 2309, + 92, 812, 812, 3050, 1812, 93, 1123, 1308, 95, 97, + 1883, 1884, 1885, 1886, 3221, 821, 3222, 812, 828, 1726, + 683, 1825, 2258, 1265, 2004, 1265, 3061, 1829, 2759, 2185, + 684, 2754, 2501, 1651, 1731, 2843, 2354, 651, 1116, 2188, + 2717, 2858, 2191, 2859, 821, 821, 21, 1576, 2845, 925, + 700, 1380, 1381, 1124, 2760, 2310, 1961, 1542, 1732, 1653, + 821, 2043, 1916, -214, -214, 699, 1991, 1592, 1593, 1594, + 1595, 1596, 1597, 1370, 1371, 2533, 1145, 721, 1713, 1937, + 3401, 2407, 1146, 724, 2513, 725, 21, 1501, 2534, 2005, + 3273, 1482, 1892, 3141, 2718, 2755, 924, 2756, 1493, 2132, + 1489, 2424, 728, 1654, 505, 505, 1496, 732, 1833, 1826, + 505, 3064, 1988, 505, 505, 919, 505, 505, 505, 505, + 1907, 2282, 73, 2363, 2679, 2301, 519, 726, 2425, 727, + 1145, 2281, 505, 2284, 2286, 2287, 1146, 1554, 1502, 505, + 3128, 1376, 1377, 1837, 2283, 2285, 2298, 101, 999, 10, + 1881, 1882, 1883, 1884, 1885, 1886, 2600, 924, 1308, 823, + 2553, 1727, 3065, 2604, 622, 2606, 3155, 1121, 2456, 1728, + 2457, 622, 1122, 1824, 984, 3248, -571, 2515, 505, 733, + 2044, -571, 1824, 1147, 628, 985, 25, 26, 27, 520, + 2184, 628, 739, 2962, 1145, 519, 505, 2045, 21, 2025, + 1146, 2542, 2046, 2265, 1713, 2792, 2009, 505, 505, 505, + 734, 505, 505, 682, 1262, -1946, -1946, 1656, 1737, 812, + 21, 1408, 1353, 2487, 1704, 2288, 25, 26, 27, 2290, + 3231, 2479, 2292, 742, 2147, 1705, 999, 1147, 1148, 1704, + 2047, 2476, 505, 1123, 1354, 743, 2426, 735, 630, 32, + 1707, 2427, -571, 1417, 744, 630, 1380, 1381, 520, 924, + 924, 2183, 821, 1910, 1435, 2234, 3281, 717, 1911, 1641, + 1642, 718, 1648, 734, 503, 698, 698, 1714, 698, 1657, + 2194, 503, 2571, 2541, 717, 2201, 37, 1353, 1553, 32, + 1124, 2548, 505, 2207, 2208, 2209, 505, 505, 2514, -571, + 2515, 1147, 2549, 2364, 3117, 2365, 505, 505, 505, 1354, + 735, 505, 102, 1512, 1417, 3118, 1313, 1486, 1546, 39, + 2063, 103, 2688, 1355, 2064, 1090, 37, 2546, 2516, 2262, + 42, 823, 737, -1947, -1947, 1765, 1479, 1148, 25, 26, + 27, 1711, 2159, -1948, -1948, 736, 2904, 2583, 988, 989, + 990, 679, 2186, 2591, 1505, 812, 104, 2189, 1509, 39, + 25, 26, 27, 1145, 523, 1525, 524, 45, 998, 1146, + 42, 1869, 1766, 732, 1653, 1870, 1871, 749, 1558, 1872, + 1873, 1874, 45, 1900, 43, 1902, 1903, 1560, 1738, 1353, + 2048, 1148, 527, 1149, 745, 737, 2681, 1150, 821, 2428, + 717, 32, -1201, 1714, 718, 1610, 105, 3056, 44, 1494, + 2429, 1354, 1409, 1499, 1615, 1410, 1417, 1417, 1654, -1951, + -1951, 108, 45, 32, 515, 1357, 750, 1151, 2171, 2171, + 2171, 563, 2248, 523, 2249, 524, 2352, 2307, 37, 1988, + 1308, 2311, -1952, -1952, 2313, 733, 649, 1175, 964, 1308, + 662, 1150, 2272, -1953, -1953, 1148, 1627, 1768, 834, 1628, + 37, 527, 2366, 748, 2367, 812, 3397, 1711, 2651, 942, + 1147, 39, 2658, 1308, 3384, 1830, 3385, 931, 1831, 925, + 38, 1176, 42, 1655, 965, 1951, 3360, 932, 1952, 1953, + 823, 1145, 1954, 39, 2526, 933, 2528, 1146, 1152, 2314, + 1487, 1772, 3372, 2746, 42, 2748, 1773, 934, 821, 1775, + 967, 2320, 812, 505, 1118, 1150, 1120, 3410, 43, 823, + 823, 937, 2903, 951, 812, -1954, -1954, 2219, 1776, 940, + 2220, 941, 1656, 2266, 45, 823, 1954, 2273, 943, 1765, + 2274, 2348, 44, 1145, 944, 1151, 945, 2783, 2353, 1146, + 2790, 946, 1152, 980, 812, 821, 45, 3408, -1956, -1956, + 2422, 812, 3409, 2126, 2127, 1840, 2523, 821, 2525, 1262, + 2275, 2747, 2302, 2274, 983, 2303, 1766, 1876, 2340, 2952, + 986, 2341, 2612, 924, 2388, 2391, 2387, 2389, 2389, 925, + 2387, 921, 2864, 991, 1657, 992, 2393, 821, 1147, 812, + 1262, -1957, -1957, 812, 821, 73, 505, -1958, -1958, 812, + 3217, 1767, 3218, 2782, 993, 2791, 1152, 1928, 2452, 1929, + 2529, 994, 1931, 2530, 1148, 505, 505, 1935, -1959, -1959, + 1938, 505, 1939, 505, 971, 1084, 1943, 1088, 505, 505, + 505, 505, 821, -1960, -1960, 1877, 821, 1827, 1099, 1828, + 1147, 1488, 821, 505, 505, 1105, 1546, 1106, 505, 2535, + 505, 1768, 2536, 505, 2594, 1108, 2663, 1952, 505, 2389, + 505, 505, 1111, 505, -1961, -1961, 1109, 505, 2602, 2603, + 1179, -1962, -1962, 2724, 1150, 628, 2725, 1112, 2876, 1138, + 622, 2220, 622, 2897, 1119, 622, 2898, 505, 505, 1140, + 622, 1142, 505, 622, 505, 622, 1143, 505, 1958, 622, + 628, 1144, 628, 1955, 1180, 628, 2567, 73, 1156, 1956, + 628, 2490, 1957, 628, 2905, 628, 1262, 2906, 2953, 628, + 505, 1954, 505, -1963, -1963, 1157, 1960, 823, 505, 505, + 505, 505, 505, 505, 505, 505, 1262, 2072, 2476, 630, + 505, 505, 1148, 1174, 3083, 505, 1839, 1954, 3084, 505, + 1512, 2389, 505, 505, 505, 505, 505, 505, 505, 505, + 505, -1965, -1965, 505, 630, 503, 630, 1178, 3112, 630, + 505, 1954, 1308, 3119, 630, 1152, 1952, 630, 3129, 630, + 1182, 3130, 2171, 630, 3082, 1311, 2846, 1310, 2175, 1315, + 503, 2107, 503, 505, 1148, 503, 1317, 698, 2146, 1096, + 503, 3162, 1150, 503, 2220, 503, 2997, 3196, 1318, 503, + 1954, 3269, -1966, -1966, 3270, 505, 2790, 1328, 3296, 3341, + 3195, 3297, 3270, 1330, 688, 3382, 505, 505, 3270, 2838, + -1967, -1967, 1151, 1331, 519, -1968, -1968, 1336, -572, 1340, + 1878, 1879, 1880, -572, 1881, 1882, 1883, 1884, 1885, 1886, + 2192, -1969, -1969, 1343, 1150, -1970, -1970, 1344, 998, -1264, + 1346, 1869, 1347, 823, 1359, 1870, 1871, 1365, 3046, -1977, + -1977, -1977, 1262, 2159, 1262, -1972, -1972, 1145, 1364, -1974, + -1974, -1976, -1976, 1146, 2193, 3359, 812, 2763, 1367, 3361, + 1756, 1757, 1395, 2764, 689, 3172, 1396, 520, 1158, -622, + -622, 1378, 1379, 1152, 2609, 1400, 2765, -626, -626, 1380, + 1381, 1403, 505, 1404, -572, 1426, 1308, 1427, 1159, 505, + 505, 1430, 2964, -625, -625, 1433, 925, 2004, 2766, 821, + 2767, 2151, 2993, 2994, 3368, 3369, 1460, 2078, 2719, 3394, + 3395, 2160, 1473, 2163, 1475, 3402, 2174, 1486, 3173, 2233, + 1621, 1623, 2178, 1308, 2180, 1152, 1476, 3174, 1483, 1490, + 505, -572, 1160, 2252, 1491, 1497, 2187, 1498, 1508, 3195, + 690, 2190, 1510, 823, 1521, 2195, 2196, 2197, 2198, 3175, + 2202, 2203, 1504, 1524, 1147, -794, 1522, 1262, -801, 1529, + 45, -647, 21, 2476, 1308, 505, -648, -791, -792, -795, + 1533, 1534, 2896, 924, -793, 1538, 1547, 1557, 1605, 1603, + 1607, 3195, -503, 1619, 1631, 1630, 1635, 2768, 1639, 1640, + 823, 1122, 1124, 1677, 505, 505, -503, 505, 1679, 1675, + 1709, -503, 823, 505, 505, 505, 505, 505, 505, 1691, + 1710, 505, 505, 505, 505, 505, 505, 505, 505, 505, + 505, 3176, 2377, 1711, 2871, 1161, 505, 505, 2380, 1716, + 505, 3195, 823, 2740, 3177, 1717, 2887, 505, 1718, 823, + 3160, 1722, 523, 1729, 524, 812, 1730, 1734, 964, 3025, + 1749, 1744, -503, 2769, 1758, 1770, 2863, 1762, 2770, 505, + 1784, 1780, 1783, 505, 1771, 505, 1786, 526, 1787, 505, + 527, 1789, -503, 1790, 1162, 1796, 1797, 823, 999, 1804, + 1806, 823, 1163, 924, 965, 1308, 691, 823, 821, 1807, + 1818, 1824, 2720, 1819, 1164, 622, 505, 1814, 1815, 1832, + 1495, 622, 25, 26, 27, 1877, 2771, 1820, 1148, 1821, + 967, 1858, 1860, 1863, 1861, 628, 1926, 1866, 1889, -503, + 1908, 628, 2772, 1897, 814, 1546, 1165, 505, -503, 3105, + 1417, 1930, 505, 505, 3250, 2841, 2306, 1898, 1905, 1936, + 1940, 1941, 1942, 2612, 1950, 1947, 1963, 1564, 812, 1994, + 1997, 1999, 3211, 2000, 2001, 2002, 505, 505, 3034, 3035, + 2021, 505, 682, 2028, 2199, 32, 2039, 2040, 1150, 2042, + 2066, 2067, 1434, 2073, 632, 505, 2085, 1167, 505, 630, + 505, 1352, 2086, 2090, 689, 630, 2103, 2106, 2118, 2115, + 2116, 821, 2117, 2119, 2120, 2141, 505, 1262, 2200, 2137, + 1168, 998, 37, 505, 1869, 503, 505, 2895, 1870, 1871, + 2142, 503, 1872, 1873, 1874, 1456, 519, 2153, 2152, 1170, + 812, 2773, 2148, 505, 2145, 2154, 690, 2217, 3178, 2682, + 2774, 3179, 2221, 2226, 971, 39, 505, 2235, 622, 2237, + 1417, -1264, 2238, 2942, 2239, 812, 42, 2253, 2254, 2256, + 2259, 1488, 2263, 505, 505, 622, 2260, 2261, 628, 2264, + 43, 2267, 2270, 821, 2291, 2268, 2269, 2271, 2279, 1152, + 505, 2280, 505, 2304, -503, 628, 1368, 1369, 2312, 520, + 815, 505, 998, 1894, 44, 1869, 1869, 2342, 821, 1870, + 1871, 1827, 1737, 1872, 1873, 1874, 1893, 2373, 2721, 1262, + 1262, 1262, 812, 2349, 2381, 2350, 2838, 2404, 2361, 2251, + -1977, -1977, -1977, 2362, 1881, 1882, 1883, 1884, 1885, 1886, + 2351, 2355, 630, 2394, 2423, 2379, 2395, 2403, 961, 21, + 2402, 521, 3347, 962, 2405, 2418, 2430, 505, 2406, 630, + 717, 2439, 2447, 1308, 718, 821, 2416, 3011, 503, 2440, + 501, 512, 2458, 1370, 1371, 533, 2445, 2446, 2459, 2461, + 533, 3232, 814, 2463, 619, 503, 633, 633, 2464, 682, + 1772, 637, 533, 645, 823, 1773, 645, 2469, 1775, 663, + 667, 2471, 2472, 667, -628, 2483, 533, 533, 2482, 2486, + 963, 2485, 505, 2489, 2493, 2495, 3033, 1776, 1668, 2494, + 2497, 2499, 2500, 2503, 2522, 2524, 522, 1456, 2537, 812, + 1876, 2543, 2544, 2554, 619, 619, 2545, 1372, 1373, 1374, + 1375, 1376, 1377, 2555, 2556, 1378, 1379, 2565, 2566, 2570, + 924, 2580, 2581, 2584, 2585, 2588, 1262, 663, 2592, 2599, + 2597, 2616, 2607, 812, 667, 533, 667, 667, 667, 2624, + 2608, 1455, 821, 2627, 523, 2630, 524, 2587, 2633, 964, + 2634, 2635, 2636, 2531, 2647, 3149, 2648, 2659, 2140, 25, + 26, 27, 2660, 525, 505, 2666, 2670, 2676, 1877, 526, + 2686, 2684, 527, 2687, 2705, 2707, 821, 2690, 2699, 2700, + 2714, 1876, 2733, 2731, 3334, 965, 2737, 925, 2745, 2778, + 2870, 2794, 2711, 2735, 2879, 2752, 2860, 2751, 2559, 2861, + 2758, 966, 2862, 2866, 505, 2874, 2875, 812, 815, 3232, + 2878, 967, 2702, 2883, 2892, 2572, 628, 2894, 2220, 2901, + 505, 2902, 32, 2907, 505, 505, 1380, 1381, 2927, 2929, + 2933, 505, 682, 2938, 2944, 2945, 2965, 2984, 2972, 2975, + 2986, 505, 814, 968, 505, 2990, 505, 2004, 2991, 1877, + 821, 2838, 2977, 823, 505, 2992, 2998, 505, 505, 37, + 2999, 3000, 505, 505, 682, 3004, 3008, 3009, 3010, 505, + 3018, 3200, 3023, 3202, 3021, 3024, -1217, 1417, 3028, 3040, + 630, 3041, -1945, -1946, 505, 2614, -1947, 2615, 969, 3042, + -1948, 2620, 39, 2623, 505, 970, 1958, -1949, 3213, 3053, + -1950, 1955, 2723, 42, -1951, 73, 503, 1956, -1952, 3212, + 1957, 3043, 3215, -1953, 505, 2739, 3051, 43, 817, -1954, + -1956, -1957, -1958, -1959, 1960, -1960, -1961, 3057, -1962, 3044, + -1963, 3071, 2841, 3210, 3073, 971, -1965, 3075, -1966, -1967, + -1968, 44, 1382, 1383, -1969, 812, 2657, 3076, -1970, -1971, + -1972, -1973, 972, 1262, -1974, 45, 2844, 1262, -1975, 1262, + 3298, -1976, -1218, 3058, 3079, 3085, 3294, 3086, 3089, 3091, + 505, 3093, 505, 1878, 1879, 1880, 3100, 1881, 1882, 1883, + 1884, 1885, 1886, 3097, 3099, 3103, 3104, 3106, 821, 3111, + 2742, 2743, 3126, 3127, 3131, 3134, 3139, 3137, 815, 3151, + 3152, 3159, 3161, 2868, 1772, 3168, 3167, 3169, 3182, 1773, + 3183, 3184, 1775, 3287, 814, 3197, 1456, 1456, 2795, 3198, + 3201, 973, 1456, 3204, 3062, 3205, 3219, 3214, 823, 2888, + 3207, 1776, 2890, 2848, 2849, 2850, 2851, 2852, 2853, 2854, + 2855, 2856, 2857, 3239, 21, 3017, 3282, 3243, 3245, 3246, + 505, 3249, 3255, 823, 1878, 1879, 1880, 3261, 1881, 1882, + 1883, 1884, 1885, 1886, 3262, 3263, 3267, 3272, 3279, 2968, + 505, 505, 1158, 3274, 3276, 505, 682, 3286, 505, 3293, + 3284, 3280, 3285, 3291, 3295, 3300, 3311, 505, 3305, 1482, + 3312, 3313, 1159, 3318, 3319, 3328, 3320, 3339, 3330, 505, + 3332, 3333, 3349, 3352, 505, 505, 3354, 3373, 3350, 505, + 823, 3370, 3351, 2837, 505, 3387, 3392, 505, 505, 3398, + 3403, 3412, 505, 1308, 3413, 719, 505, 3226, 2520, 3326, + 505, 2179, 1110, 2793, 2726, 3290, 1160, 505, 3396, 2293, + 3322, 3031, 2559, 3309, 3147, 682, 3380, 2797, 3348, 3355, + 3374, 667, 3181, 2502, 2527, 667, 817, 2882, 667, 3346, + 3353, 2498, 622, 1701, 2762, 533, 3344, 2847, 1822, 1800, + 815, 1769, 2562, 814, 2613, 3080, 1632, 2488, 1480, 1481, + 2865, 505, 628, 2474, 25, 26, 27, 2841, 1672, 505, + 2144, 1671, 2638, 826, 533, 533, 3331, 2484, 2112, 3275, + 812, 3203, 814, 814, 1158, 2143, 623, 2460, 505, 1676, + 3108, 1447, 2631, 2385, 936, 2936, 2114, 823, 814, 2632, + 2657, 2662, 3153, 950, 1159, 1813, 2300, 3278, 1436, 1161, + 3277, 2414, 927, 512, 633, 2477, 1438, 1610, 1442, 501, + 1443, 667, 2437, 821, 2668, 2415, 630, 32, 2398, 2435, + 619, 2844, 1867, 1868, 954, 954, 1917, 2976, 1888, 954, + 977, 3012, 2678, 2712, 3265, 2392, 929, 2081, 1160, 2784, + 1856, 2727, 503, 645, 645, 645, 1834, 2785, 1162, 2083, + 537, 706, 0, 0, 37, 0, 1163, 645, 645, 0, + 645, 2974, 0, 505, 0, 0, 0, 0, 1164, 0, + 0, 0, 0, 667, 0, 0, 0, 0, 0, 533, + 0, 0, 0, 0, 0, 0, 3063, 39, 0, 0, + 667, 924, 0, 0, 505, 823, 0, 0, 42, 815, + 1165, 0, 0, 0, 667, 2786, 0, 0, 0, 0, + 505, 682, 43, 0, 0, 0, 817, 0, 0, 0, + 0, 0, 0, 0, 0, 818, 0, 0, 815, 815, + 1308, 1161, 0, 0, 0, 0, 44, 505, 667, 667, + 667, 667, 0, 0, 815, 0, 0, 0, 505, 505, + 45, 1167, 505, 0, 505, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1158, 0, 0, 0, + 0, 0, 0, 0, 1168, 0, 0, 0, 0, 505, + 1162, 0, 0, 0, 1563, 820, 1159, 1564, 1163, 0, + 814, 1565, 1566, 1170, 0, -1977, -1977, -1977, 0, 0, + 1164, 0, 2787, 505, 1456, 1456, 1456, 1456, 1456, 1456, + 0, 0, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, + 1456, 1456, 1574, 0, 0, 3059, 0, 0, 0, 1575, + 1160, 0, 1165, 2844, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1417, 0, + 0, 0, 0, 0, 1576, 0, 0, 0, 0, 0, + 3164, 3165, 0, 998, 0, 0, 1869, 0, 505, 0, + 1870, 1871, 505, 0, 1872, 1873, 1874, 0, 0, 0, + 0, 0, 505, 1167, 505, 0, 505, 3150, 817, 0, + 505, 2955, 505, 0, 505, 0, 0, 2786, 0, 0, + 0, 0, 0, 0, 505, 1972, 1168, 3157, 3158, 505, + 0, 505, 1973, 1974, 0, 0, 1975, 1976, 1977, 505, + 0, 0, 3124, 1161, 628, 1170, 814, 2788, 3170, 0, + 2789, 0, 1262, 0, 3223, 0, 3225, 505, 954, 977, + 0, 667, 0, 0, 1454, 0, 815, 0, 1577, 0, + 954, 954, 0, 0, 0, 0, 533, 1456, 1456, 0, + 0, 0, 619, 818, 0, 1578, 0, 0, 0, 619, + 1579, 0, 1162, 0, 0, 3257, 0, 0, 0, 533, + 1163, 0, 0, 0, 505, 0, 1511, 0, 630, 0, + 2837, 3260, 1164, 0, 505, 3264, 1523, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 505, 1582, 0, + 0, 0, 0, 0, 503, 0, 0, 0, 0, 0, + 0, 505, 0, 820, 1165, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 814, 1711, 0, 505, + 2321, 2322, 2323, 2324, 2325, 2326, 533, 817, 2330, 2331, + 2332, 2333, 2334, 2335, 2336, 2337, 2338, 2339, 0, 0, + 0, 1585, 0, 505, 0, 0, 0, 0, 0, 0, + 0, 1456, 1876, 628, 0, 1167, 817, 817, 0, 505, + 671, 0, 815, 814, 0, 0, 0, 0, 2844, 0, + 1262, 1624, 817, 1626, 998, 814, 0, 1869, 1168, 0, + 0, 1870, 1871, 0, 0, 1872, 1873, 1874, 533, 533, + 0, 3251, 0, 505, 667, 628, 0, 1170, 0, 3356, + 0, 0, 2956, 0, 0, 814, 0, 0, 505, 505, + 505, 0, 814, 0, 0, 0, 1454, 630, 0, 0, + 1877, 708, 0, 711, 0, 715, 3371, 667, 1700, 0, + 0, 0, 0, 818, 0, 505, 0, 3340, 1587, 0, + 667, 0, 0, 503, 0, 628, 0, 0, 0, 0, + 814, 0, 0, 0, 814, 0, 0, 667, 0, 630, + 814, 667, 2475, 0, 0, 1746, 67, 0, 0, 0, + 0, 0, 815, 2419, 2420, 0, 505, 21, 0, 0, + 0, 0, 0, 998, 67, 503, 1869, 621, 0, 0, + 1870, 1871, 0, 820, 1872, 1873, 1874, 0, 0, 0, + 0, 0, 67, 0, 0, 0, 0, 0, 0, 630, + 0, 681, 0, 0, 0, 0, 0, 0, 0, 815, + 0, 0, 0, 0, 0, 2837, 0, 0, 0, 0, + 0, 815, 0, 0, 0, 503, 0, 621, 621, 696, + 0, 0, 0, 1588, 0, 0, -1977, -1977, -1977, 0, + 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, 0, 0, + 67, 815, 3124, 0, 0, 0, 0, 0, 815, 0, + 0, 0, 0, 0, 817, 1145, 0, 0, 0, 1368, + 1369, 1146, 0, 1876, 0, 0, 0, 0, 0, 1158, + 0, 0, 0, 0, 1759, 0, 667, 0, 0, 0, + 0, 667, 0, 0, 0, 818, 815, 3224, 0, 1159, + 815, 0, 1793, 0, 0, 0, 815, 25, 26, 27, + 0, 0, 0, 0, 0, 1878, 1879, 1880, 0, 1881, + 1882, 1883, 1884, 1885, 1886, 0, 0, 0, 0, 0, + 0, 0, 1810, 0, 0, 0, 0, 1810, 1456, 1456, + 0, 1877, 0, 1160, 0, 0, 1370, 1371, 0, 0, + 0, 0, 0, 0, 0, 820, 0, 1978, 1979, 1980, + 0, 1981, 1982, 1983, 1984, 1985, 1986, 0, 826, 0, + 32, 826, 1147, 533, 533, 0, 533, 826, 0, 0, + 0, 0, 1876, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 21, 0, + 817, 0, 0, 1511, 0, 0, 0, 37, 0, 0, + 1372, 1373, 1374, 1375, 1376, 1377, 0, 0, 1378, 1379, + 0, 0, 0, 0, 0, 1454, 1454, 0, 0, 0, + 534, 1454, 0, 501, 0, 534, 1161, 814, 0, 0, + 39, 0, 0, 0, 0, 0, 954, 534, 533, 1912, + 1877, 42, 0, 0, 818, 0, 667, 0, 619, 0, + 619, 534, 534, 619, 1145, 43, 0, 0, 619, 0, + 1146, 619, 0, 619, 0, 0, 0, 619, 1158, 533, + 0, 533, 0, 818, 818, 1162, 0, 0, 0, 44, + 0, 0, 0, 1163, 0, 0, 0, 0, 1159, 818, + 0, 0, 0, 45, 0, 1164, 1987, 0, 0, 0, + 817, 1993, 0, 0, 820, 0, 0, 0, 0, 0, + 534, 0, 0, 0, 0, 0, 1148, 0, 723, 1380, + 1381, 1456, 730, 0, 0, 731, 0, 1165, 25, 26, + 27, 0, 1160, 820, 820, 0, 1878, 1879, 1880, 826, + 1881, 1882, 1883, 1884, 1885, 1886, 2930, 817, 0, 820, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 817, + 0, 1147, 0, 0, 2692, 2693, 0, 0, 0, 0, + 2948, 0, 1166, 0, 0, 0, 1150, 0, 1167, 0, + 0, 0, 0, 815, 0, 0, 0, 0, 2079, 817, + 667, 32, 667, 0, 0, 0, 817, 0, 0, 0, + 0, 1168, 667, 2095, 0, 0, 1169, 0, 939, 0, + 0, 0, 0, 0, 1843, 1454, 814, 0, 0, 0, + 1170, 0, 0, 0, 0, 1161, 0, 0, 37, 0, + 0, 0, 0, 0, 817, 1382, 1383, 0, 817, 0, + 667, 0, 533, 0, 817, 1878, 1879, 1880, 0, 1881, + 1882, 1883, 1884, 1885, 1886, 1384, 0, 1700, 533, 1844, + 1456, 39, 0, 67, 0, 0, 0, 0, 533, 2161, + 533, 2165, 42, 533, 1162, 0, 0, 1152, 1845, 533, + 0, 533, 1163, 0, 0, 0, 43, 0, 0, 0, + 0, 818, 826, 533, 1164, 0, 1846, 826, 533, 0, + 1847, 1139, 533, 533, 533, 533, 0, 533, 533, 0, + 44, 67, 681, 0, 0, 1148, 0, 0, 0, 814, + 0, 0, 1848, 0, 45, 1849, 1165, 0, 0, 667, + 667, 667, 667, 667, 0, 1320, 1322, 1325, 1327, 0, + 0, 1850, 0, 0, 0, 0, 0, 2241, 0, 0, + 0, 820, 935, 0, 0, 0, 0, 2900, 0, 0, + 0, 0, 0, 947, 0, 0, 0, 0, 0, 0, + 0, 2181, 815, 0, 0, 1150, 0, 1167, 0, 0, + 0, 0, 0, 0, 0, 0, 0, -1709, 0, 0, + 0, 814, 0, 0, 0, 1456, 0, 0, 0, 0, + 1168, 0, 0, 0, 0, 2182, 0, 0, 0, 0, + 0, 696, 961, 0, 0, 961, 814, 962, 0, 1170, + 962, 1563, 0, 0, 1564, 0, 0, 818, 1565, 1566, + 67, 0, 1851, 1454, 1454, 1454, 1454, 1454, 1454, 0, + 1852, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, + 1454, 0, 0, 0, 0, 0, 0, 0, 0, 1574, + 0, 0, 1853, 0, 0, 0, -1977, 0, 0, 0, + 0, 533, 0, 814, 963, 815, 1152, 963, 0, 0, + 0, 0, 0, 667, 0, -1709, 0, 820, 1854, 0, + 0, 1576, 0, 619, 0, 0, 3013, 0, 0, 619, + 0, 0, 0, 0, 0, 533, 0, 0, 0, 0, + 533, 0, 0, 1563, 0, 0, 1564, 0, 0, 0, + 1565, 1566, 0, 0, 0, 0, 0, 0, 0, 0, + -1709, 817, 0, 0, 0, 0, 0, 818, 1449, 0, + 534, 0, 0, 964, -1709, 0, 964, 815, 0, -1709, + 0, 1574, 0, 0, -1709, 533, 0, 0, -1977, 2411, + 0, 0, 0, -1709, 533, 0, 0, 0, -1709, 534, + 534, 0, 815, 0, 0, 0, 0, 0, 0, 965, + 814, 0, 965, 1576, 818, -1977, 1454, 1454, 0, 0, + 0, 0, 0, 3209, 0, 966, 818, 820, 966, 0, + -1709, 0, -1977, 1987, 0, 967, 0, -1977, 967, 0, + 0, 0, 0, 0, 814, 0, 0, 0, 1368, 1369, + -1709, 0, 0, 0, 0, 0, 818, 0, 0, 815, + 0, 3081, 0, 818, 0, 0, 619, 968, 0, 0, + 968, 0, 0, 0, 820, -1977, 2421, 0, 533, 0, + 0, 0, 0, 619, 0, 0, 820, 2095, 1932, 0, + 0, 1135, 0, 0, 0, 0, 0, -1709, 0, 0, + -1709, 818, 67, 0, 0, 818, -1709, -1977, 0, 0, + 0, 818, 969, 0, 534, 969, 820, 0, 814, 970, + 0, 0, 970, 820, -1977, 1370, 1371, 533, 1585, -1977, + 1454, 0, 0, 533, 0, 621, 0, 0, 0, 0, + 0, 0, 621, 0, 0, -1709, 0, 0, 0, 1759, + 817, 0, 0, 0, 0, 0, 0, 0, 0, 971, + 0, 820, 971, 0, 0, 820, 815, -1977, -1709, 0, + 0, 820, 0, 0, 1692, 0, 972, 1526, 0, 972, + 0, 0, 0, 0, 0, 0, 0, 1719, 1530, 1372, + 1373, 1374, 1375, 1376, 1377, 0, 0, 1378, 1379, 0, + 815, 0, 0, 0, 0, 0, 0, 0, 1743, 0, + 0, 0, 0, 1759, 0, 1544, 0, 0, 0, 667, + 1585, 0, 0, 0, 669, -1977, 0, 0, 0, 0, + 1759, 667, 667, 667, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 533, 973, 667, 0, 973, 667, + 1933, 0, -1709, 817, 667, 0, 814, 0, 0, 0, + 0, 0, -1709, 0, 0, 0, 0, 2605, 0, 0, + 0, 0, 0, 0, 815, 0, 0, 0, 0, -1709, + 0, -1709, -1709, 0, 826, 0, 0, 0, 0, 0, + 0, 0, 0, 696, 696, 0, 696, 0, 0, 0, + 0, 0, 1759, 1759, 0, 1759, 0, 0, 1380, 1381, + 0, 0, 0, 0, 0, 0, 0, -1977, -1709, 0, + 0, -1709, -1709, -1709, 0, 817, 0, 0, -1711, 0, + -1977, 0, 0, 0, 501, 0, 0, 1592, 1593, 1594, + 1595, 1596, 1597, 1764, 0, 0, 0, 0, 1779, 0, + 817, 0, 0, 0, 2637, 0, 0, 0, 0, 0, + 0, 0, 667, 667, 667, 0, 0, 0, 0, 0, + 0, 0, 533, 0, 1454, 0, 533, 0, 0, 0, + 0, 0, 533, 0, 0, 0, 0, 0, 0, 0, + 0, 1474, 0, 0, 1457, 0, 0, 1987, 818, 0, + 0, 0, 0, 0, 0, 0, 0, 817, 0, 0, + 0, 0, 815, 0, 1507, 0, 0, 0, 0, 0, + 0, 0, -1977, 0, 1382, 1383, -1711, 1454, 1454, 1592, + 1593, 1594, 1595, 1596, 1597, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 815, 0, 0, 969, 0, 0, 0, 815, 0, - 0, -1968, 0, 0, 0, 0, 0, 1873, 0, 0, - 0, 814, 0, 0, 0, 0, 531, 531, 0, 531, - 0, 0, 0, 0, 1756, 0, 664, 1582, 0, 0, - 0, 664, 0, 0, 1446, 0, 815, 0, 0, 0, - 815, 0, 1790, 0, 0, 814, 815, 1584, 0, 0, - 0, 0, 0, 0, 1582, 0, 0, 693, 693, 0, - 693, 0, 970, 0, 0, 0, 0, 1930, 0, 0, - 0, 0, 1807, 0, 0, 1874, 0, 1807, 0, 0, - 0, 0, 3056, 0, 0, 0, 817, 0, 0, 0, - 995, 531, 0, 1866, 0, 0, 0, 1867, 1868, 0, - 0, 1869, 1870, 1871, 0, 1414, 0, 0, 823, 0, - 0, 823, 0, 530, 530, 0, 530, 823, 2679, 814, - 0, 0, 1941, -1968, 1945, 0, 0, 0, 0, 0, - 0, 0, 0, 817, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1508, 0, 817, 0, 0, 0, 0, - -1968, 1585, 0, 0, -1968, -1968, -1968, 0, 1589, 1590, - 1591, 1592, 1593, 1594, 0, 1451, 1451, 0, 0, 0, - 0, 1451, 0, 498, 0, 817, 0, 0, 0, 3121, - 0, 0, 817, 0, 0, 0, 951, 0, 530, 1909, - 0, 0, 0, 0, 0, 0, 664, 0, 616, 0, - 616, 0, 0, 616, 0, 0, 0, 0, 616, 0, - 0, 616, 0, 616, 0, 0, 0, 616, 0, 530, - 817, 530, 0, 0, 817, 0, 0, -1968, 0, 0, - 817, 0, 0, 21, 1589, 1590, 1591, 1592, 1593, 1594, - 1689, 0, 0, 0, 0, 0, 1984, 2834, 21, 1417, - 0, 1990, 0, 1716, -1968, 0, 0, 814, 0, 0, - 0, 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, 0, - 0, 0, 0, 0, 1740, 0, 0, 0, 1875, 1876, - 1877, 0, 1878, 1879, 1880, 1881, 1882, 1883, 0, 823, - 0, 0, 0, 0, 0, 531, 0, 0, 1873, 0, + 0, 0, 0, 0, 0, 0, 2716, 0, 820, 0, + 0, 0, 2095, 0, 0, 0, 0, 0, 0, 0, + 0, -1711, 0, 0, 0, 0, 0, 0, 667, 0, + 0, 1549, 533, 0, 0, -1711, 533, 533, 533, 0, + -1711, 0, 1759, 1700, 1759, -1711, 1793, 0, 0, 0, + 0, 0, 0, 0, -1711, 0, 0, 0, 0, -1711, + 0, 814, 0, 1924, 817, 533, 0, 2796, 1563, 0, + 0, 1564, 0, 0, 0, 1565, 1566, 0, 0, 0, + 533, 533, 533, 533, 533, 533, 533, 533, 533, 533, + 0, -1711, 0, 534, 534, 0, 0, 0, 817, 0, + 0, 0, 0, 0, 0, 0, 1574, 0, 2241, 0, + 0, -1711, 0, -1977, 667, 0, 0, 0, 0, 1420, + 0, 0, 0, 0, 1700, 0, 0, 0, 961, 0, + 0, 0, 0, 962, 0, 67, 1457, 818, 1576, 0, + 1793, 0, 998, 0, 538, 1869, 0, 0, 0, 1870, + 1871, 0, 0, 1872, 1873, 1874, 0, 0, -1711, 1759, + 0, -1711, 0, 0, 0, 0, 0, -1711, 0, 0, + 1454, 0, 817, 0, 0, 0, 533, 0, 0, 0, + 1420, 0, 0, 667, 667, 667, 667, 0, 0, 0, + 963, 0, 0, 0, 0, 1454, 0, 820, 1454, 539, + 0, 2431, 533, 826, 0, 0, -1711, 2080, 0, 2082, + 0, 621, 0, 621, 0, 540, 621, 815, 2946, 2092, + 0, 621, 0, 0, 621, 0, 621, 0, 0, -1711, + 621, 0, -1977, 0, 0, 0, 0, 0, 0, 0, + 818, 0, 0, 0, 0, 0, 0, 67, 2969, -1977, + 619, 0, 0, 1561, -1977, 0, 0, 2129, 0, 964, + 0, 0, 541, 0, 2981, 0, 0, 1132, 2095, 0, + 0, 0, 542, 0, 1987, 0, 0, 0, 0, 0, + 1420, 0, 1420, 1420, 543, 669, 0, 0, 1700, 544, + 0, 0, -1977, 0, 1759, 965, 1563, 0, 0, 1564, + 820, 0, 0, 1565, 1566, 2029, 0, 826, 533, 1454, + 817, 966, 818, -1711, 0, 667, 0, 0, 545, 0, + 0, 967, 0, -1711, 0, 0, 0, 0, 0, 0, + 0, 0, 3032, 0, 1574, 0, 0, 818, 0, 0, + -1711, -1977, -1711, -1711, 0, 1585, 2227, 2228, 2230, 2231, + 2232, -1977, 0, 968, 0, 1420, 0, 0, 1420, 1420, + 0, 546, 0, 0, 0, 547, 1576, 0, 0, 0, + 0, 0, 820, 0, 1133, 696, 0, 0, 0, -1711, + 0, 0, -1711, -1711, -1711, 0, 0, 0, 534, 534, + 0, 534, 0, 0, 818, 0, 0, 820, 969, 0, + 0, 0, 0, 0, 0, 970, 0, 0, 0, 1759, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1877, + 0, 2241, 0, 0, 0, 0, 0, 0, 0, 2691, + 0, 0, 548, 0, 0, 3077, 0, 0, 0, 0, + 0, 0, -1977, 0, 1454, 971, 0, 549, 0, 0, + 0, 0, 0, 0, 820, 1457, 1457, 0, 0, 0, + -1977, 1457, 972, 534, 0, 0, 0, 0, 0, 0, + 3094, 0, 0, 0, 0, 0, 533, -1977, 0, 0, + 550, 0, -1977, 551, 533, 0, 0, 0, 0, 0, + 0, 0, 552, 0, 1944, 553, 1948, 0, 0, 0, + 2368, 818, 0, 2236, 0, 0, 0, 0, 0, 0, + 0, 3121, 67, 554, 0, 0, 0, 0, 0, 0, + -1977, 0, 0, 0, 0, 0, 0, 555, 0, 0, + 0, 973, 0, 533, 556, 818, 1134, 0, 533, 0, + 1799, 0, 0, 0, 557, 0, 0, -1977, 0, 0, + 558, 1808, 0, 1809, 1592, 1593, 1594, 1595, 1596, 1597, + 0, 820, 0, 1816, 533, 817, 0, 0, 0, 0, + 0, 0, 0, 1585, 0, 0, 0, 559, 0, 0, + 1817, 0, 0, 0, 533, 533, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 820, 0, 0, 0, 667, + 0, 1700, 0, 0, 0, 533, 0, 0, 2071, 818, + 0, 1836, 1838, 0, 0, 0, 0, 0, 0, 667, + 0, 0, 3192, 0, 1878, 1879, 1880, 0, 1881, 1882, + 1883, 1884, 1885, 1886, 0, 0, 0, 0, 0, 0, + 0, 0, 1454, 0, 0, 0, 533, 954, 0, 0, + 954, 0, 0, 0, 0, 1457, 621, 0, 0, 0, + 0, 0, 621, 0, 0, 0, 0, 534, 0, 820, + -1977, 0, 0, 2981, 0, 0, 0, 0, 0, 0, + 667, 0, 0, 534, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 534, 0, 534, 0, 0, 534, 0, + 0, 0, 0, 0, 534, 0, 534, 0, 0, 0, + 667, 0, 0, 0, 0, 0, 0, 0, 534, 0, + 0, 0, 0, 534, 0, 0, 0, 534, 534, 534, + 534, 0, 534, 534, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 818, 3094, 0, + 0, 0, 0, 0, 1544, 0, 0, 0, 0, 0, + 0, 1058, 1058, 0, 0, 0, 0, 0, 0, 0, + 0, 3192, 2029, 0, 0, -1977, 0, 0, 0, 0, + 0, 0, 1592, 1593, 1594, 1595, 1596, 1597, 2441, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1700, 621, + 0, 0, 0, 0, 0, 0, 2563, 820, 0, 0, + 826, 826, 0, 3192, 826, 0, 621, 0, 2576, 2577, + 2579, 0, 1912, 0, -1726, 0, 1184, 0, 0, 0, + 1303, 0, 0, 2590, 533, 0, 2593, 0, 0, 0, + 0, 2598, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1700, 0, 0, 2124, + 2124, 0, 0, 3192, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1457, 1457, 1457, 1457, 1457, 1457, 0, + 0, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, + 1457, 0, 0, 0, 0, 0, 534, 0, 0, 0, + 1563, 826, 0, 1564, 0, 0, 0, 1565, 1566, 1567, + 1568, 1569, 1570, 1571, 0, 0, 2095, 0, 0, 1420, + 0, 0, -1726, 0, 0, 0, 0, 0, 1572, 1420, + 2384, 0, 1420, 0, 1563, 1948, 0, 1564, 1574, 0, + 0, 1565, 1566, 1407, 0, 1575, 0, 0, 0, 2644, + 2645, 2646, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 681, 0, 0, 0, 0, 0, -1726, 0, 0, + 1576, 0, 1574, 0, 0, 0, 0, 0, 0, -1977, + 534, -1726, 0, 0, 0, 0, -1726, 0, 0, 1549, + 0, -1726, 0, 0, 0, 0, 0, 0, 0, 0, + -1726, 0, 818, 0, 1576, -1726, 1563, 0, 0, 1564, + 2276, 2278, 0, 1565, 1566, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1457, 1457, 0, 0, + 0, 0, 0, 0, 0, 0, 1420, -1726, 0, 0, + 1431, 0, 0, 0, 1574, 0, 1448, 0, 0, 1450, + 0, -1977, 1461, 1464, 1469, 1472, 0, -1726, 0, 0, + 0, 0, 820, 0, 1577, 2734, 0, 2694, 0, 0, + 0, 0, 0, 534, 0, 0, 1576, 0, 0, 0, + 0, 1578, 0, 0, 0, 0, 1579, 0, 0, 0, + 0, 0, 0, 0, 1303, 0, 0, 0, -1977, 0, + 0, 0, 0, 0, -1726, 0, 0, -1726, 0, 1580, + 1581, 0, 0, -1726, 1527, -1977, 0, 0, 0, 0, + -1977, 0, 534, 0, 1582, 0, 0, 0, 2492, 0, + 0, 0, 1532, 0, 1544, 0, 0, 0, 0, 2695, + 1457, 0, 0, 1535, 1536, 1537, 1420, 1541, 1545, 0, + 0, 2873, -1726, 0, 0, 0, 0, 0, -1977, 0, + 1583, 0, 0, 1584, 0, 0, 1544, 0, 0, 0, + -1977, 0, 0, 0, 0, -1726, 0, 1585, 1606, 0, + 1586, 0, 0, 0, 0, 67, 0, -1977, 0, 0, + 0, 67, -1977, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1417, 531, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 531, 0, 531, 0, 0, 531, 0, 0, 0, - 0, 0, 531, 815, 531, 0, 1560, 0, 2076, 1561, - 664, 0, 664, 1562, 1563, 0, 531, 0, 0, 67, - 0, 531, 664, 2092, 0, 531, 531, 531, 531, 0, - 531, 531, 0, 0, 0, 1451, 1874, 0, 3248, 0, - 0, 25, 26, 27, 1571, 0, 3221, 0, 0, 0, - 0, -1968, 0, 1558, 0, 0, 25, 26, 27, 0, - 664, 0, 530, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1573, 1697, 530, 1761, - 1417, 0, 1417, 1417, 1776, 0, 0, 0, 530, 2158, - 530, 2162, 0, 530, 0, 618, 0, 618, 0, 530, - 618, 530, 0, 0, 32, 618, 0, 0, 618, 0, - 618, 0, 823, 530, 618, 0, 0, 823, 530, 32, - 0, 0, 530, 530, 530, 530, 0, 530, 530, 0, - 0, 67, 0, 0, 0, 0, 0, 0, 0, 2691, - 0, 37, 0, 0, 0, 0, 0, 0, 0, 664, - 664, 664, 664, 664, 0, 1417, 37, 0, 1417, 1417, - 0, 0, 0, 0, 0, 0, 0, 2238, 0, 0, - -1968, 0, 2834, 39, 0, 0, 0, 0, 0, 0, - 0, 0, 814, 0, 42, 0, -1968, 817, 39, 2026, - 0, -1968, 815, 0, 531, 0, 0, 0, 43, 42, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 3121, - 0, 0, 0, 43, 0, 0, 0, 0, 0, 0, - 0, 0, 44, 0, 0, 0, 0, 0, 2381, -1968, - 0, 0, 0, 1945, 0, 0, 45, 44, 0, 1875, - 1876, 1877, 0, 1878, 1879, 1880, 1881, 1882, 1883, 1921, - 0, 45, 0, 1451, 1451, 1451, 1451, 1451, 1451, 693, - 0, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, - 1451, 0, 0, 0, 0, -1703, 0, 0, 531, 0, - 0, 0, 1582, 0, 0, 0, 0, 1546, 0, 0, - 0, 530, 1454, 0, 0, 815, 0, 1155, 0, 0, - 0, 0, 0, 664, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 616, 0, 0, 0, 1156, 0, 616, - 0, 0, 0, 0, 0, 530, 0, 0, 0, 0, - 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1796, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1805, 0, 1806, 0, 0, 0, 0, 0, 0, - 0, 1157, 2781, 1813, 0, 0, 0, 815, 0, 0, - 2782, 531, 0, -1703, 0, 530, 817, 0, -1968, 2408, - 1814, 0, 0, 0, 530, 0, 0, 2233, 0, 0, - 0, 0, 815, 2077, 0, 2079, 67, 0, 0, 0, - 0, 0, 0, 0, 0, 2089, 1451, 1451, 0, 0, - 0, 1833, 1835, 0, 0, 0, 958, 0, -1703, 0, - 531, 959, 0, 1984, 0, 0, 2489, 1560, 2783, 0, - 1561, 0, -1703, 0, 1562, 1563, 0, -1703, 0, 0, - 0, 0, -1703, 2126, 0, 0, 0, 0, 0, 815, - 0, -1703, 0, 1158, 0, -1703, 616, 0, 0, 0, - 0, 0, 0, 0, 0, 1571, 0, 0, 530, 0, - 0, 0, -1968, 616, 0, 0, 0, 2092, 960, 817, - 0, 0, -1968, 0, 1454, 0, 0, -1703, 0, 1589, - 1590, 1591, 1592, 1593, 1594, 0, 0, 1573, 0, 0, - 0, 0, 1159, 0, 0, 0, 0, -1703, 0, 0, - 1160, 0, 0, 0, 0, 0, 0, 530, 0, 0, - 1451, 0, 1161, 530, 2784, 0, 0, 0, 0, 0, - 0, 0, 2224, 2225, 2227, 2228, 2229, 531, 0, 1756, - 618, 0, 0, 0, 0, 0, 618, 961, 0, 0, - 0, 817, 0, 1162, -1703, 0, 815, -1703, 0, 0, - 2692, 1560, 0, -1703, 1561, 0, 0, 0, 1562, 1563, - 0, 0, 0, 0, 0, 0, 817, 0, 0, 0, - 0, 0, 962, 0, 0, 0, 0, 0, 0, 0, - 815, -1968, 0, 0, 0, 0, 0, 0, 963, 1571, - 0, -1703, 0, 1756, 1164, 0, -1968, -1968, 964, 664, - 0, 0, -1968, 0, 0, 0, 0, 0, 0, 0, - 1756, 664, 664, 664, -1703, 0, 0, 1165, 0, 0, - 0, 1573, 0, 817, 530, 0, 664, 0, 1541, 664, - 965, 0, 0, 0, 664, 0, 1167, 0, 2785, 0, - -1968, 2786, 0, 0, 0, 0, 2026, 2602, 0, 2121, - 2121, 1132, 0, 0, 815, 531, 0, 0, 0, 531, - 0, 0, 2438, 0, 823, 1941, 0, 0, 0, 0, - 666, 0, 0, 618, 966, 0, 2365, 0, 0, 0, - 0, 967, 1756, 1756, 2956, 1756, 0, 0, 0, 0, - 618, 0, 0, 1582, 0, 0, 0, -1703, 0, 0, - 0, 0, 0, 0, 0, 0, 0, -1703, 0, 1417, - 0, 0, 0, 0, 498, -1968, 0, 0, 0, 1417, - 0, 968, 1417, 0, -1703, 0, -1703, -1703, 0, 0, - 817, -1968, 0, 0, 2634, 0, -1968, 0, 969, 0, - 0, 0, 664, 664, 664, 0, 0, 1708, 0, 0, - 0, 0, 530, 0, 1451, 0, 530, 0, 0, 0, - 0, 0, 530, -1703, 817, 531, -1703, -1703, -1703, 531, - 531, 531, 0, 0, -1968, 0, 0, 1984, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, -1968, - 0, 0, 815, 1454, 1454, 0, 0, 0, 531, 1454, - 2273, 2275, 0, 0, 0, 0, 970, 1451, 1451, 0, - 0, 0, 0, 531, 531, 531, 531, 531, 531, 531, - 531, 531, 531, 0, 0, 0, 1417, 1582, 0, 0, - 0, 0, 0, 0, 0, 678, 2713, 0, 817, 0, - 0, 0, 2092, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 664, 0, - 0, 0, 530, 0, 0, 0, 530, 530, 530, 0, - 0, 0, 1756, 1697, 1756, 0, 1790, 0, 0, 0, + 0, 1585, 0, 0, 0, 0, 0, 2738, 0, 0, + 2909, 2910, 2911, 2912, 0, 0, 0, 0, 1646, 0, + -1977, 669, 1662, 1667, 0, 0, 0, 0, 0, 534, + 0, 0, 0, 1058, 1058, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, -1726, + 0, 0, 0, 0, 0, 0, 0, 0, 0, -1726, + 0, 0, 0, 2454, 1587, 0, 0, 0, 0, 0, + 0, 0, 0, 1585, 0, 0, -1726, 0, -1726, -1726, + 0, 0, 67, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2480, 2480, -1977, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 67, 0, + 0, 67, 0, 0, 0, -1726, 0, 0, -1726, -1726, + -1726, 0, 0, 0, 0, 0, 1563, 0, 0, 1564, + 0, 0, 0, 1565, 1566, 1567, 1568, 1569, 1570, 1571, + 0, 0, 3019, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1572, 0, 0, 534, 0, 0, + 0, 534, 0, 0, 1574, 0, 1303, 1944, 1544, 1588, + -1977, 1575, 1589, 1590, 1591, 1303, 1592, 1593, 1594, 1595, + 1596, 1597, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2401, 1576, 0, 0, 1303, + 0, 0, 0, -1977, 0, 0, 0, 0, 0, 0, + 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, 0, 0, + 0, 0, 0, 621, 0, 0, 0, 1457, 1457, 0, + 0, 0, 0, 0, 0, 0, 0, 681, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, -1968, 0, 530, 0, 2793, 0, 0, - 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, 0, 1941, - 530, 530, 530, 530, 530, 530, 530, 530, 530, 530, - 0, 0, 0, -1968, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2381, 1417, 0, 2238, 0, - 0, 0, 0, 0, 664, 0, 2068, 0, 0, 0, - 0, 0, 0, 995, 1697, 0, 1866, 0, 0, 0, - 1867, 1868, 0, 0, 1869, 1870, 1871, 0, 0, 0, - 1790, 0, 0, 0, 0, 0, 817, 0, 0, 0, - 0, 2952, 2560, 0, 0, 0, 0, 0, 0, 1756, - 0, 1055, 1055, 1454, 2573, 2574, 2576, 0, 0, 0, - 1451, 0, 0, 0, 0, 0, 530, 0, 1541, 2587, - 0, 0, 2590, 664, 664, 664, 664, 2595, 0, 0, - 0, 0, -1717, 0, 0, 1451, 0, -1968, 1451, 0, - 0, 0, 530, 823, 1589, 1590, 1591, 1592, 1593, 1594, - 1541, 531, 0, 0, 0, 0, 0, 815, 2943, 0, - 0, 0, 0, 0, 0, 0, 1181, 0, 0, 67, - 1300, 0, 0, 2451, 995, 67, 0, 1866, 0, 0, - 0, 1867, 1868, 0, 0, 1869, 1870, 1871, 2966, 0, - 616, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 2735, 2953, 0, 2978, 0, 2477, 2477, 2092, 0, - 995, 0, 0, 1866, 1984, 0, 0, 1867, 1868, 0, - 0, 1869, 1870, 1871, 0, 0, 0, 0, 1697, 0, - -1717, 0, 0, 0, 1756, 2641, 2642, 2643, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 823, 530, 1451, - 0, 0, 0, 0, 0, 664, 995, 0, 0, 1866, - 0, 0, 0, 1867, 1868, 0, 0, 1869, 1870, 1871, - 0, 1873, 3029, 0, 0, -1717, 67, 0, 0, 0, - 0, 0, 0, 1404, 0, 0, 0, 0, 0, -1717, - 0, 0, 0, 0, -1717, 0, 0, 0, 0, -1717, - 0, 0, 67, 0, 0, 67, 0, 0, -1717, 0, - 0, 0, -1717, 0, 0, 0, 0, 0, 0, 3099, - 0, 1454, 1454, 1454, 1454, 1454, 1454, 2381, 0, 1454, - 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1874, - 0, 817, 0, 0, -1717, 0, 0, 0, 0, 1756, - 0, 2731, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 2238, 1541, 2598, -1717, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 3074, 531, 0, 0, 0, - 1428, 531, 1873, 0, 1451, 0, 1445, 0, 0, 1447, - 0, 0, 1458, 1461, 1466, 1469, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 531, 0, 0, - 3091, -1717, 0, 0, -1717, 0, 530, 618, 1873, 0, - -1717, 0, 0, 0, 530, 0, 0, 531, 531, 0, - 0, 678, 0, 0, 1300, 0, 0, 2870, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 531, 0, - 1874, 3118, 0, 0, 1524, 0, 0, 0, -1717, 0, - 0, 0, 0, 0, -1968, 0, 0, 0, 0, 0, - 0, 0, 1529, 530, 1454, 1454, 0, 0, 530, 0, - 0, -1717, 0, 1532, 1533, 1534, 1874, 1538, 1542, 531, + 0, 0, 0, 2601, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 534, 0, 0, + 0, 534, 534, 534, 0, -1977, 0, 0, 0, 0, + 0, 0, 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, + 1577, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 534, 0, 0, 0, 0, 0, 0, 1578, 0, 0, + 0, 0, 1579, 0, 0, 534, 534, 534, 534, 534, + 534, 534, 534, 534, 534, 0, 0, 0, 0, 0, + 0, 0, 1859, 0, 0, 1580, 1581, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 530, 0, 2906, 2907, 2908, 2909, - 0, 0, 0, 0, 0, 0, 0, 0, 1603, 0, - 0, 0, 0, 0, 530, 530, 0, 0, 0, 0, - 0, 0, 1874, 0, 0, 0, 0, 666, 0, 664, - 0, 1697, 1875, 1876, 1877, 530, 1878, 1879, 1880, 1881, - 1882, 1883, 0, 0, 0, 0, 0, 0, 0, 664, - 1365, 1366, 3189, 0, -1717, 0, 0, 0, 1643, 0, - 0, 0, 1659, 1664, -1717, 0, 0, 0, 0, 0, - 0, 0, 1451, 1055, 1055, 0, 530, 951, 1454, 67, - 951, -1717, 0, -1717, -1717, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1560, 0, 0, 1561, 0, 0, - 0, 1562, 1563, 2978, 0, 1566, 1567, 1568, 0, 0, - 664, 0, 0, 0, 0, 0, 0, 1541, 0, 1417, - -1717, 0, 1569, -1717, -1717, -1717, 1367, 1368, 3016, 0, - 0, 0, 1571, 0, 0, 0, 0, 0, 0, 1572, - 664, 0, 0, 1875, 1876, 1877, 0, 1878, 1879, 1880, - 1881, 1882, 1883, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1573, 0, 0, 531, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3091, 1875, - 1876, 1877, 0, 1878, 1879, 1880, 1881, 1882, 1883, 0, - 1369, 1370, 1371, 1372, 1373, 1374, 0, 0, 1375, 1376, - 0, 3189, 0, 0, 0, 0, 1300, 0, 0, 0, - 0, 0, 0, 0, 0, 1300, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1875, 1876, 1877, 1697, 1878, - 1879, 1880, 1881, 1882, 1883, 0, 0, 0, 0, 1300, - 823, 823, 0, 3189, 823, 0, 0, 0, 0, 0, - 0, 0, 1909, 67, 67, 0, 0, 0, 1574, 0, - 0, 0, 0, 0, 530, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1575, 0, 0, 0, 0, 1576, - 1560, 0, 0, 1561, 0, 0, 1697, 1562, 1563, 0, - 0, 0, 0, 3189, 0, 0, 0, 0, 0, 0, - 0, 0, 1577, 1578, 0, 0, 0, 0, 1377, 1378, - 0, 0, 0, 0, 0, 0, 0, 1579, 1571, 0, - 0, 0, 0, 0, 0, -1968, 0, 67, 0, 67, - 823, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2092, 0, 0, 0, 0, 0, - 1573, 0, 0, 1580, 0, 0, 1581, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 67, 1417, - 1582, 0, 1856, 1583, 1417, 1454, 1454, 0, 0, 0, - 0, 0, 0, 0, 67, 0, 0, 0, 67, 0, + 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1469, 0, 1469, 1469, 0, 0, + 1563, 0, 0, 1564, 0, 0, 0, 1565, 1566, 1058, + 1058, 1569, 1570, 1571, 0, 0, 1583, 0, 0, 1584, + 0, 0, 0, 0, 0, 67, 1058, 0, 1572, 0, + 0, 0, 0, 1585, 0, 0, 1586, 0, 1574, 0, + 0, 1944, 0, 0, 0, 1575, 0, 0, 0, 0, + 1457, 0, 0, 0, 0, 0, 3166, 0, 1966, 0, + 1967, 0, 0, 1544, 0, 0, 0, 2384, 0, 0, + 1576, 0, 0, 0, 0, 1457, 3190, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 2006, 0, + 0, 0, 0, 0, 2010, 2011, 2012, 2013, 2014, 2015, + 2016, 2017, 0, 0, 0, 0, 2026, 2027, 0, 0, + 0, 2038, 0, 0, 0, 2041, 0, 0, 2049, 2050, + 2051, 2052, 2053, 2054, 2055, 2056, 2057, 0, 0, 2058, + 1587, 0, 0, 0, 0, 0, 1058, 3230, 1303, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1420, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2084, + 0, 0, 0, 0, 1577, 0, 0, 3256, 0, 0, + 0, 0, 0, 534, 0, 0, 0, 0, 0, 0, + 0, 1578, 0, 0, 0, 0, 1579, 0, 0, 1457, + 0, 0, 1536, 1537, 0, 0, 1458, 0, 0, 67, + 67, 0, 0, 0, 0, 0, 0, 0, 0, 1580, + 1581, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1582, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1588, 0, 0, 1589, 1590, + 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, + 0, 0, 0, 0, 0, 1862, 0, 0, 0, 0, + 1583, 0, 0, 1584, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 67, 0, 67, 0, 1585, 2205, 0, + 1586, 0, 1303, 0, 0, 2215, 2216, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1466, 0, 1466, 1466, 0, 0, - 0, 0, 3163, 0, 0, 0, 0, 0, 0, 1055, - 1055, 1833, 1835, 1379, 1380, 0, 0, 0, 0, 0, - 0, 0, 3187, 0, 0, 0, 1055, 0, 0, 0, - 0, 0, 0, 1381, -1968, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - -1968, 0, 0, 0, 0, -1968, 1584, 0, 1963, 0, - 1964, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1600, 67, 0, 0, 0, 0, 1303, + 0, 0, 0, 0, 1457, 0, 0, 0, 0, 0, + 67, 0, 1563, 0, 67, 1564, 0, 0, 0, 1565, + 1566, 3102, 0, 1569, 1570, 1571, 0, 0, 0, 2384, + 0, 0, 0, 0, 0, 0, 1600, 0, 0, 0, + 1407, 2294, 0, 0, 0, 0, 0, 0, 0, 0, + 1574, 0, 0, 0, 1587, 0, 1563, 1575, 0, 1564, + 0, 0, 0, 1565, 1566, 0, 0, 0, 1458, 0, + 2316, 2317, 0, 2318, 0, 0, 0, 0, 534, 1420, + 0, 0, 1576, 534, 1420, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1574, 0, 0, 0, 0, 0, + 0, -1977, 2344, 2345, 0, 0, 2084, 0, 0, 534, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1600, + 0, 0, 0, 0, 0, 0, 1576, 0, 0, 534, + 534, 1836, 1838, 0, 0, 2372, 0, 0, 67, 0, + 0, 2378, 0, 0, 0, 0, 0, 0, 0, 0, + 534, 0, 0, 0, 0, 0, 0, 0, 0, 1588, + 0, 1303, 1589, 1590, 1591, 67, 1592, 1593, 1594, 1595, + 1596, 1597, 2396, 0, 0, 0, 1577, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2959, + 0, 534, 0, 1578, 0, 0, 0, 0, 1579, 0, + 0, 0, 1457, 2409, 0, 0, 0, 0, 2412, 2413, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3227, 0, 0, 0, 0, 2003, 0, - 0, 0, 0, -1968, 2007, 2008, 2009, 2010, 2011, 2012, - 2013, 2014, 67, 0, 0, 0, 2023, 2024, 0, 0, - 0, 2035, 0, 3253, 0, 2038, 0, 0, 2046, 2047, - 2048, 2049, 2050, 2051, 2052, 2053, 2054, 0, 67, 2055, - 0, 0, 0, 0, 0, 0, 1055, 0, 1300, 0, - 0, 0, 0, 0, 0, 0, 1582, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1454, 2081, - 1585, 0, 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, - 1592, 1593, 1594, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1454, 0, 0, 0, 0, 0, 0, - 0, 0, 1533, 1534, 0, 0, 0, 0, 0, 0, - -42, 0, 0, 0, 0, 1833, 1835, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1417, 0, 0, 0, - 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 2, 0, 3, 4, 0, 0, 0, 0, - 0, 0, -1968, 0, 0, 0, 0, 5, 0, 0, - 0, 0, 6, 0, 0, 0, 0, 0, 0, 0, - 0, 7, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 8, 0, 0, 0, 0, 2202, 0, - 0, 0, 1300, 0, 9, 2212, 2213, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 10, 1454, 11, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1455, 12, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1300, - 0, 0, 0, 13, 14, 0, 0, 0, 0, 0, - 0, 0, 0, 15, 0, 0, 0, 0, 1142, 16, - 0, 0, 0, 0, 1143, 0, -1968, 17, 0, 18, - 19, 0, 1155, 1589, 1590, 1591, 1592, 1593, 1594, 0, - 1404, 2291, 0, 20, 0, 0, 0, 21, 0, 0, - 0, 0, 1156, 0, 0, 0, 1560, 0, 0, 1561, - 0, 0, 0, 1562, 1563, 1564, 1565, 1566, 1567, 1568, - 2313, 2314, 22, 2315, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1569, 0, 0, 0, 23, 0, - 0, 0, 0, 0, 1571, 0, 1157, 0, 0, 0, - 0, 1572, 2341, 2342, 0, 1597, 2081, 0, 0, 0, - 0, 0, 1454, 0, 24, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1144, 1573, 0, 0, 0, - 0, 0, 0, 0, 0, 2369, 0, 0, 0, 0, - 0, 2375, 0, 0, 0, 0, 0, 0, 1597, 0, - 0, 1417, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1300, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 2393, 0, 0, 0, 0, 0, 0, 0, - 1455, 0, 0, 0, 0, 25, 26, 27, 1158, 0, - 0, 0, 0, 28, 0, 0, 29, 0, 0, 0, - 0, 0, 0, 2406, 0, 0, 0, 0, 2409, 2410, + -1977, -1977, -1977, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1582, -1977, 1600, 0, + 0, 0, -1977, 0, 0, 1562, 0, 0, 0, 0, + 1563, 2433, 0, 1564, 2436, 0, 2438, 1565, 1566, 1567, + 1568, 1569, 1570, 1571, 0, 0, 0, 0, 0, 0, + 0, 0, 2442, 0, 0, -1977, 0, 0, 1572, 0, + -1977, 0, 1573, 0, 0, 0, 0, 0, 1574, 1585, + 0, 0, 0, 0, 0, 1575, 0, 0, 0, 0, + 0, 0, 1563, 0, 0, 1564, 0, 0, 0, 1565, + 1566, 0, 1646, 1569, 1570, 1571, 0, 0, 0, 0, + 1576, 0, 0, 0, 0, 1836, 1838, 0, 0, 1667, + 2056, 0, 0, 1585, 0, 0, 1420, 0, 0, 0, + 1574, 0, 1600, 0, 0, 0, 0, 1575, 1058, 0, + 0, 0, 0, 0, 0, 0, 0, 2496, 0, 0, + 0, 0, 0, 0, 0, 0, 1600, 0, 0, 0, + 0, 0, 1576, 0, 0, 0, 0, 0, 0, 534, + 0, 0, 0, 1600, 0, 1600, 1587, 1458, 1458, 0, + 1887, 0, 0, 1458, 0, 0, 1600, 0, 0, 1600, + 0, 0, 0, 0, 1600, 0, 0, 1600, 0, 0, + 0, 0, 0, 1537, 1577, 0, 0, 0, 0, 1303, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1574, 0, 0, 0, 0, 30, 0, 0, 0, 0, - 0, 1597, 0, 0, 31, 0, 1575, 1159, 0, 0, - 0, 1576, 0, 0, 0, 1160, 0, 0, 32, 0, - 0, 2430, 0, 0, 2433, 33, 2435, 1161, 0, 34, - 0, 0, 0, 0, 1577, 1578, 0, 0, 0, 35, - 0, 0, 2439, 0, 0, 0, 0, 0, 1145, 1579, - 0, 36, 0, 0, 0, 37, 0, 0, 1162, 0, + -1977, 1578, 0, 0, 0, 0, 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 38, 0, 0, 0, 535, 0, - 1454, 0, 1643, 0, 0, 1580, 0, 39, 1581, 0, - 40, 0, 0, 41, 0, 0, 0, 0, 42, 1664, - 2053, 0, 1582, 1163, 0, 1583, 0, 1147, 0, 1164, - 0, 0, 43, 0, 0, 0, 0, 0, 1055, 0, - 1597, 0, 0, 0, 0, 0, 0, 2493, 0, 0, - 0, 0, 1165, 536, 0, 0, 44, 1166, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 537, - 45, 1167, 1560, -42, 0, 1561, 0, 0, 0, 1562, - 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1580, + 1581, 0, 1600, 0, 0, 0, 1577, 1600, 2596, 0, + 1600, 1600, 1600, 0, 1582, 0, 1600, 0, 0, 0, + 1600, 0, 0, 1578, 0, 0, 0, 0, 1579, 0, + 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, 1593, + 1594, 1595, 1596, 1597, 0, 0, 0, 0, 0, 0, + 1583, 1580, 1581, 1584, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1582, 1585, 0, 0, + 1586, 0, 0, 0, 0, -1977, 0, 0, 0, 0, + 0, 1600, 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1569, 0, 0, 1534, 0, 0, 0, 0, 0, 1300, - 1571, 0, 0, 0, 0, 0, 538, 1572, 1584, 0, - 0, 0, 0, 0, 0, 0, 539, 1149, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 540, 0, - 0, 0, 1573, 541, 1597, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 2593, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1597, 0, - 0, 542, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1597, 0, 1597, 0, 1455, - 1455, 0, 1884, 0, 0, 1455, 0, 0, 1597, 0, - 0, 1597, 0, 0, 0, 0, 1597, 0, 0, 1597, - 0, 0, 0, 0, 543, 0, 0, 0, 544, 0, - 0, 0, 1585, 0, 0, 1586, 1587, 1588, 0, 1589, - 1590, 1591, 1592, 1593, 1594, 0, 1574, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 2398, 0, - 0, 0, 1575, 0, 0, 0, 0, 1576, 0, 0, - 0, 0, 0, 0, 1597, 0, 0, 0, 0, 1597, - 0, 0, 1597, 1597, 1597, 0, 0, 0, 1597, 0, - 1577, 1578, 1597, 0, 545, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1579, 2662, 0, 0, 546, - 2664, 1964, 0, 0, 0, 0, 0, 2668, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 2677, 0, 0, - 2680, 0, 2682, 0, 0, 0, 0, 0, 0, 0, - 2686, 1580, 547, 0, 1581, 548, 0, 0, 2693, 2694, - 0, 0, 0, 1597, 549, 2701, 0, 550, 1582, 0, - 0, 1583, 0, 0, 0, 0, 0, 0, 0, 0, - 2710, 0, 0, 0, 0, 551, 0, 0, 0, 0, - 2725, 0, 0, 0, 0, 0, 0, 0, 0, 552, - 0, 0, 0, 1597, 0, 0, 553, 0, 0, 0, - 1055, 0, 0, 0, 0, 554, 0, 0, 0, 1597, - 0, 555, 0, 0, 1597, 0, 0, 0, 0, 0, + 0, 0, 1583, 0, 0, 1584, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1585, + 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1420, 0, 0, 0, 0, 0, 1600, 0, 0, + 0, 0, 1600, 0, 0, 0, 2665, 0, 0, 0, + 2667, 1967, 0, 0, 0, 0, 0, 2671, 0, 1887, + 0, 0, 0, 0, 1587, 0, 0, 2680, 0, 0, + 2683, 0, 2685, 0, 0, 0, 0, 0, 0, 0, + 2689, 0, 0, 0, 0, 0, 0, 0, 2696, 2697, + 0, 0, 0, 0, 0, 2704, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1884, 0, 0, 0, 0, 0, 0, 556, 0, + 2713, 0, 0, 0, 0, 0, 1587, 0, 0, 0, + 2728, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1584, 1559, 2202, 0, 2202, 0, - 1560, 0, 0, 1561, 0, 0, 0, 1562, 1563, 1564, - 1565, 1566, 1567, 1568, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1569, 0, - 0, 0, 1570, 0, 0, 0, 0, 0, 1571, 0, - 0, 0, 0, 0, 0, 1572, 0, 1560, 0, 0, - 1561, 0, 0, 0, 1562, 1563, 1564, 1565, 1566, 1567, - 1568, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1573, 0, 0, 0, 0, 1569, 0, 0, 0, 2672, - 0, 0, 0, 0, 0, 1571, 0, 0, 0, 0, - 0, 0, 1572, 0, 0, 0, 2922, 2923, 1585, 0, - 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, - 1594, 0, 0, 2940, 0, 0, 0, 1573, 1859, 0, - 0, 0, 0, 0, 0, 2944, 0, 0, 0, 0, - 2946, 2947, 0, 0, 0, 2948, 0, 0, 0, 0, - 2951, 0, 0, 2954, 2955, 0, 1597, 0, 2202, 1300, - 0, 0, 2963, 0, 1884, 1884, 0, 1455, 1455, 1455, - 1455, 1455, 1455, 1055, 1574, 1455, 1455, 1455, 1455, 1455, - 1455, 1455, 1455, 1455, 1455, 1884, 0, 0, 0, 0, - 1575, 0, 0, 0, 0, 1576, 0, 0, 0, 0, - 1560, 0, 0, 1561, 0, 0, 0, 1562, 1563, 1564, - 1565, 1566, 1567, 1568, 0, 0, 0, 3004, 1577, 1578, - 0, 1574, 0, 0, 0, 0, 0, 0, 1569, 0, - 0, 0, 0, 1579, 0, 0, 0, 1575, 1571, 0, - 0, 0, 1576, 0, 3023, 1572, 0, 0, 0, 0, + 1058, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1597, 1597, 1577, 1578, 0, 0, 1580, - 1573, 0, 1581, 0, 0, 0, 0, 0, 0, 0, - 1579, 0, 0, 0, 0, 0, 1582, 0, 0, 1583, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1588, + 0, 0, 1589, 1590, 1591, 0, 1592, 1593, 1594, 1595, + 1596, 1597, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2205, 0, 2205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1597, 0, 0, 0, 1597, 1597, 1597, - 1597, 1597, 1597, 1597, 1597, 0, 1580, 0, 0, 1581, - 1455, 1455, 0, 1597, 1597, 0, 0, 0, 0, 3052, - 0, 0, 0, 1582, 0, 1597, 1583, 0, 1597, 0, - 0, 0, 0, 0, 0, 0, 1597, 1597, 1597, 1597, - 1597, 1597, 1597, 1597, 1597, 1597, 0, 0, 0, 0, - 0, 0, 0, 0, 1574, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1542, 0, 0, 0, - 1575, 1597, 1584, 0, 0, 1576, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1300, 0, 0, 0, - 0, 0, 0, 2406, 0, 0, 0, 0, 1577, 1578, - 0, 0, 0, 0, 3110, 3111, 0, 0, 3112, 0, - 1534, 0, 0, 1579, 0, 0, 0, 0, 0, 1584, - 0, 0, 0, 0, 1455, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 3130, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1580, - 0, 0, 1581, 0, 0, 0, 0, 0, 0, 3142, - 0, 0, 0, 0, 0, 0, 1582, 0, 0, 1583, - 0, 0, 0, 0, 0, 0, 1585, 0, 0, 1586, - 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, 1594, 0, - 0, 0, 1597, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1597, 1597, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1585, 3186, 0, 1586, 1587, 1588, 0, - 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, 0, 0, - 1995, 0, 2922, 0, 0, 0, 3203, 0, 0, 0, - 1055, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 3213, 0, 1584, 0, 0, 2202, 0, 2202, 0, 0, - 0, 0, 0, 0, 0, 1055, 0, 0, 0, 0, - 0, 1597, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3238, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1597, 1597, 1597, 0, 0, 1884, 1884, - 1884, 1884, 1884, 1884, 0, 0, 0, 1884, 1884, 1884, - 1884, 1884, 1884, 1884, 1884, 1884, 1884, 0, 0, 0, - 0, 1597, 1597, 0, 1560, 0, 0, 1561, 0, 0, - 3263, 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, - 2922, 0, 0, 0, 0, 0, 0, 0, 0, 1597, - 0, 0, 1569, 1055, 0, 1597, 1585, 0, 0, 1586, - 1587, 1588, 1571, 1589, 1590, 1591, 1592, 1593, 1594, 1572, - 0, 1745, 0, 1597, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 3307, 1597, 0, 0, 1597, - 1597, 0, 0, 0, 1573, 0, 1884, 1884, 0, 0, + 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, 1593, + 1594, 1595, 1596, 1597, 1600, 0, 0, 0, 0, 0, + 0, 0, 1887, 1887, 0, 1458, 1458, 1458, 1458, 1458, + 1458, 0, 0, 1458, 1458, 1458, 1458, 1458, 1458, 1458, + 1458, 1458, 1458, 1887, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1597, 1455, 1455, 1597, 0, 1597, 0, 0, 0, 1597, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2925, 2926, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 2943, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2947, 0, 0, 0, 0, + 2949, 2950, 0, 0, 0, 2951, 0, 0, 0, 0, + 2954, 1600, 1600, 2957, 2958, 0, 0, 0, 2205, 1303, + 0, 0, 2966, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1058, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1142, 0, 0, 0, 0, 0, 1143, 0, - 0, 0, 0, 0, 0, 0, 1155, 0, 0, 1560, - 0, 0, 1561, 1597, 3361, 3361, 1562, 1563, 1564, 1565, - 1566, 1567, 1568, 0, 0, 0, 1156, 0, 1574, 0, - 0, 0, 0, 0, 0, 0, 0, 1569, 0, 0, - 3361, 0, 0, 0, 1575, 0, 0, 1571, 0, 1576, - 0, 0, 0, 0, 1572, 0, 0, 0, 0, 0, + 0, 1600, 0, 0, 0, 1600, 1600, 1600, 1600, 1600, + 1600, 1600, 1600, 0, 0, 0, 0, 0, 1458, 1458, + 0, 1600, 1600, 0, 0, 0, 0, 3007, 0, 0, + 0, 0, 0, 1600, 0, 0, 1600, 0, 0, 0, + 0, 0, 0, 0, 1600, 1600, 1600, 1600, 1600, 1600, + 1600, 1600, 1600, 1600, 3026, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1157, 0, 1577, 1578, 0, 0, 3361, 0, 0, 1573, - 0, 0, 0, 0, 0, 0, 0, 1579, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1560, 0, 1144, - 1561, 0, 0, 0, 1562, 1563, 1564, 1565, 1566, 1567, - 1568, 0, 0, 1597, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1580, 0, 1569, 1581, 0, 0, 0, - 0, 0, 0, 0, 0, 1571, 0, 0, 0, 0, - 1582, 0, 1572, 1583, 1455, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1158, 0, 0, 0, 0, 1573, 0, 0, - 0, 0, 0, 1574, 0, 0, 0, 0, 0, 0, - 0, 0, 1597, 0, 1597, 0, 0, 0, 1597, 1575, - 0, 0, 0, 0, 1576, 0, 0, 1597, 0, 0, - 1597, 0, 1597, 0, 0, 0, 1597, 0, 0, 1884, - 1884, 1159, 0, 1597, 1597, 0, 0, 1577, 1578, 1160, - 0, 1597, 0, 0, 0, 0, 0, 0, 0, 0, - 1597, 1161, 1579, 0, 0, 0, 1584, 0, 0, 0, - 0, 0, 0, 0, 0, 1597, 0, 0, 0, 0, - 0, 0, 1145, 0, 0, 0, 0, 0, 0, 0, - 0, 1574, 1162, 0, 0, 0, 0, 0, 1580, 0, - 0, 1581, 0, 0, 0, 0, 0, 1575, 0, 0, - 0, 0, 1576, 1455, 0, 1582, 0, 0, 1583, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1577, 1578, 2178, 0, 0, - 0, 1147, 0, 1164, 0, 0, 0, 0, 0, 0, - 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1165, 0, 0, 0, - 1585, 2179, 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, - 1592, 1593, 1594, 0, 0, 1167, 1580, 1995, 0, 1581, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1582, 0, 0, 1583, 0, 0, 0, - 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1149, 0, 0, 0, 0, 0, 1884, 1455, 0, + 0, 0, 1458, 0, 0, 0, 0, 0, 0, 3055, + 0, 0, 0, -42, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2, 0, 3, 4, 0, + 0, 0, 0, 0, 0, 0, 1545, 0, 0, 0, + 5, 0, 0, 0, 0, 6, 0, 0, 0, 0, + 1600, 0, 0, 0, 7, 0, 1303, 0, 0, 0, + 1600, 1600, 0, 2409, 0, 0, 8, 0, 0, 0, + 0, 0, 0, 0, 3113, 3114, 0, 9, 3115, 0, + 1537, 0, 0, 0, 0, 0, 0, 0, 0, 10, + 0, 11, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 12, 0, 0, 3133, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 13, 14, 0, 0, + 0, 0, 0, 0, 0, 0, 15, 0, 0, 3145, + 0, 0, 16, 0, 0, 0, 0, 0, 0, 1600, + 17, 0, 18, 19, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 20, 0, 0, 0, + 21, 1600, 1600, 1600, 0, 0, 1887, 1887, 1887, 1887, + 1887, 1887, 0, 0, 0, 1887, 1887, 1887, 1887, 1887, + 1887, 1887, 1887, 1887, 1887, 0, 22, 0, 0, 1600, + 1600, 0, 0, 0, 3189, 0, 0, 0, 0, 0, + 0, 0, 23, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 2925, 0, 0, 0, 3206, 1600, 0, 0, + 1058, 0, 0, 1600, 0, 0, 0, 0, 24, 0, + 3216, 0, 0, 0, 0, 2205, 0, 2205, 0, 0, + 0, 1600, 0, 0, 0, 1058, 0, 0, 0, 0, + 0, 0, 0, 0, 1600, 0, 0, 1600, 1600, 0, + 0, 0, 0, 3241, 1887, 1887, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1600, 1458, + 1458, 1600, 0, 1600, 0, 0, 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1597, 1597, 0, 0, 0, 0, 0, 0, + 25, 26, 27, 0, 0, 0, 0, 0, 28, 0, + 3266, 29, 0, 0, 0, 0, 0, 0, 0, 0, + 2925, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1058, 0, 0, 0, 0, 0, 0, + 30, 1600, 0, 1563, 0, 0, 1564, 0, 0, 31, + 1565, 1566, 1567, 1568, 1569, 1570, 1571, 0, 0, 0, + 0, 0, 0, 32, 0, 3310, 0, 0, 0, 0, + 33, 1572, 0, 0, 34, 2675, 0, 0, 0, 0, + 0, 1574, 0, 0, 35, 0, 0, 0, 1575, 0, + 0, 0, 0, 0, 0, 0, 36, 0, 0, 0, + 37, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1576, 0, 1563, 0, 0, 1564, 0, + 38, 0, 1565, 1566, 1567, 1568, 1569, 1570, 1571, 0, + 0, 0, 0, 39, 0, 0, 40, 0, 0, 41, + 0, 1600, 0, 1572, 42, 0, 0, 0, 0, 0, + 0, 0, 0, 1574, 3365, 3365, 3365, 0, 43, 0, + 1575, 0, 0, 1563, 0, 0, 1564, 0, 0, 0, + 1565, 1566, 1458, 0, 1569, 1570, 1571, 0, 0, 0, + 0, 3365, 44, 0, 0, 1576, 0, 0, 0, 0, + 0, 1572, 0, 0, 0, 0, 45, 0, 0, -42, + 0, 1574, 0, 0, 0, 0, 0, 1577, 1575, 0, + 1600, 0, 1600, 0, 0, 0, 1600, 0, 0, 0, + 0, 0, 3365, 0, 1578, 1600, 0, 0, 1600, 1579, + 1600, 0, 0, 1576, 1600, 0, 0, 1887, 1887, 0, + 0, 1600, 1600, 0, 0, 0, 0, 0, 0, 1600, + 0, 0, 1580, 1581, 0, 0, 0, 0, 1600, 0, + 0, 0, 0, 0, 0, 0, 0, 1582, 0, 0, + 0, 0, 0, 1600, 0, 0, 0, 0, 0, 1577, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1597, 0, 0, 0, 1597, 0, 1597, 1597, 1597, 1584, - 0, 1597, 0, 0, 1597, 1597, 0, 0, 0, 0, - 0, 0, 0, 1597, 0, 1585, 0, 0, 1586, 1587, - 1588, 0, 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, - 0, 0, 2312, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1578, 0, 0, 0, + 0, 1579, 0, 1583, 0, 0, 1584, 0, 0, 0, + 0, 1458, 0, 0, 0, 0, 0, 0, 0, 0, + 1585, 0, 0, 1586, 1580, 1581, 0, 1577, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1582, + 0, 0, 0, 0, 1578, 0, 0, 0, 0, 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1597, 0, 0, 0, 0, 0, - 1884, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1597, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1580, 1581, 0, 1583, 0, 0, 1584, 0, + 0, 0, 0, 0, 0, 0, 0, 1582, 0, 0, + 0, 0, 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1597, 1585, 0, 0, 1586, 1587, 1588, 0, - 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, 0, 0, - 2397, 0, 0, 0, 0, 0, 0, 0, 1884, 0, + 0, 0, 0, 0, 0, 0, 0, 1587, 0, 0, + 0, 0, 0, 1583, 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1585, 0, 0, 0, 0, 1887, 1458, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1597, 1597, 1597, 0, 0, 0, 0, 0, 0, 0, + 1600, 1600, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1600, 1587, + 0, 0, 1600, 0, 1600, 1600, 1600, 0, 0, 1600, + 0, 0, 1600, 1600, 0, 0, 0, 0, 0, 0, + 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, + 1593, 1594, 1595, 1596, 1597, 0, 0, 1587, 0, 1998, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1597, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1597, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1600, 0, 0, 0, 0, 0, 1887, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1588, 0, 0, 1589, 1590, 1591, + 1600, 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, 0, + 0, 1998, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1887, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1597, 0, 0, 0, + 0, 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, + 1593, 1594, 1595, 1596, 1597, 0, 0, 0, 1600, 1600, + 1600, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1597, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1597, 0, 0, 0, 0, 0, 0, + 1600, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1597, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1597, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 109, 992, 629, 993, 994, 995, 996, - 997, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, - 0, 118, 119, 120, 0, 0, 0, 1597, 0, 998, - 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, - 128, 129, 130, 131, 999, 133, 1000, 1001, 0, 136, - 137, 138, 139, 140, 141, 1002, 599, 142, 143, 1003, - 1004, 146, 0, 147, 148, 149, 150, 600, 0, 601, - 0, 1005, 154, 155, 156, 157, 158, 159, 160, 161, - 162, 1597, 163, 164, 165, 166, 167, 168, 0, 169, - 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, - 1006, 1007, 182, 1008, 183, 0, 184, 185, 186, 187, - 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, - 195, 196, 764, 198, 199, 0, 200, 201, 202, 0, - 203, 204, 205, 0, 206, 207, 208, 209, 1009, 211, - 212, 213, 214, 215, 602, 1010, 217, 0, 218, 219, - 1011, 221, 0, 222, 0, 223, 224, 0, 225, 226, - 227, 228, 229, 230, 0, 231, 0, 1012, 1013, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 255, 256, 257, 258, 259, 260, 261, - 1014, 1015, 0, 1016, 0, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 277, - 278, 279, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 1017, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 1018, 308, 1019, 310, 311, 312, 313, 1020, 314, 315, - 316, 317, 1021, 604, 319, 1022, 321, 322, 323, 0, - 324, 325, 0, 0, 1023, 327, 328, 0, 0, 329, - 330, 331, 332, 333, 606, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 607, 348, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 1024, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 384, 385, 386, 387, 1025, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 410, 411, 412, 1026, 414, 0, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 609, 430, 0, 431, 432, 0, 433, 434, 435, 436, - 437, 438, 439, 0, 1027, 1028, 0, 0, 442, 443, - 610, 445, 611, 1029, 447, 448, 612, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 1030, - 0, 469, 470, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 1031, 0, 0, 0, 0, 0, - 0, 1032, 1033, 1034, 0, 0, 0, 0, 1035, 0, - 1036, 0, 0, 0, 0, 1037, 1038, 1039, 1040, 0, - 2544, 109, 992, 629, 993, 994, 995, 996, 997, 0, + 0, 0, 0, 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, - 119, 120, 0, 0, 0, 0, 0, 998, 0, 0, - 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, - 130, 131, 999, 133, 1000, 1001, 0, 136, 137, 138, - 139, 140, 141, 1002, 599, 142, 143, 1003, 1004, 146, - 0, 147, 148, 149, 150, 600, 0, 601, 0, 1005, - 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 1006, 1007, - 182, 1008, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, - 764, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 1009, 211, 212, 213, - 214, 215, 602, 1010, 217, 0, 218, 219, 1011, 221, - 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, - 229, 230, 0, 231, 0, 1012, 1013, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 255, 256, 257, 258, 259, 260, 261, 1014, 1015, - 0, 1016, 0, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 277, 278, 279, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 1017, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 1018, 308, - 1019, 310, 311, 312, 313, 1020, 314, 315, 316, 317, - 1021, 604, 319, 1022, 321, 322, 323, 0, 324, 325, - 0, 0, 1023, 327, 328, 0, 0, 329, 330, 331, - 332, 333, 606, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 607, - 348, 349, 350, 351, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 1024, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 384, 385, 386, 387, 1025, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 0, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, - 412, 1026, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 427, 428, 609, 430, - 0, 431, 432, 0, 433, 434, 435, 436, 437, 438, - 439, 0, 1027, 1028, 0, 0, 442, 443, 610, 445, - 611, 1029, 447, 448, 612, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 467, 1030, 0, 469, - 470, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 1031, 0, 0, 0, 0, 0, 0, 1032, - 1033, 1034, 0, 0, 0, 0, 1035, 0, 1036, 0, - 0, 0, 0, 1037, 1038, 1039, 1040, 0, 3113, 109, - 992, 629, 993, 994, 995, 996, 997, 0, 0, 0, + 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1600, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 109, 995, + 632, 996, 997, 998, 999, 1000, 1600, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, + 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, + 0, 1600, 0, 0, 1001, 0, 0, 121, 122, 123, + 0, 124, 125, 126, 127, 128, 129, 130, 131, 1002, + 133, 1003, 1004, 0, 136, 137, 138, 139, 140, 141, + 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, 148, + 149, 150, 603, 0, 604, 1600, 1008, 154, 155, 156, + 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, + 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, + 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, + 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, + 192, 193, 194, 0, 0, 195, 196, 767, 198, 199, + 1600, 200, 201, 202, 0, 203, 204, 205, 0, 206, + 207, 208, 209, 1012, 211, 212, 213, 214, 215, 605, + 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, + 223, 224, 0, 225, 226, 227, 228, 229, 230, 0, + 231, 0, 1015, 1016, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 243, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 256, + 257, 258, 259, 260, 261, 262, 1017, 1018, 0, 1019, + 0, 266, 267, 268, 269, 270, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 278, 279, 280, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 1020, 290, 291, + 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 1021, 309, 1022, 311, + 312, 313, 314, 1023, 315, 316, 317, 318, 1024, 607, + 320, 1025, 322, 323, 324, 0, 325, 326, 0, 0, + 1026, 328, 329, 0, 0, 330, 331, 332, 333, 334, + 335, 609, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 610, 350, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 1027, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 1028, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + 1029, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 612, 432, 0, + 433, 434, 0, 435, 436, 437, 438, 439, 440, 441, + 0, 442, 1030, 1031, 0, 0, 445, 446, 613, 448, + 614, 1032, 450, 451, 615, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 470, 1033, 0, 472, + 473, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 1034, 0, 0, 0, 0, 0, 0, 1035, + 1036, 1037, 0, 0, 0, 0, 1038, 0, 1039, 0, + 0, 0, 0, 1040, 1041, 1042, 1043, 0, 2547, 109, + 995, 632, 996, 997, 998, 999, 1000, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, + 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, + 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, 140, + 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, + 148, 149, 150, 603, 0, 604, 0, 1008, 154, 155, + 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 1012, 211, 212, 213, 214, 215, + 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, + 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, + 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 256, 257, 258, 259, 260, 261, 262, 1017, 1018, 0, + 1019, 0, 266, 267, 268, 269, 270, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 278, 279, 280, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 1021, 309, 1022, + 311, 312, 313, 314, 1023, 315, 316, 317, 318, 1024, + 607, 320, 1025, 322, 323, 324, 0, 325, 326, 0, + 0, 1026, 328, 329, 0, 0, 330, 331, 332, 333, + 334, 335, 609, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 610, + 350, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 1027, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 1028, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 612, 432, + 0, 433, 434, 0, 435, 436, 437, 438, 439, 440, + 441, 0, 442, 1030, 1031, 0, 0, 445, 446, 613, + 448, 614, 1032, 450, 451, 615, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 1033, 0, + 472, 473, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 1034, 0, 0, 0, 0, 0, 0, + 1035, 1036, 1037, 0, 0, 0, 0, 1038, 0, 1039, + 0, 0, 0, 0, 1040, 1041, 1042, 1043, 0, 3116, + 109, 995, 632, 996, 997, 998, 999, 1000, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, + 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, + 120, 0, 0, 0, 0, 0, 1001, 0, 0, 121, + 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, + 131, 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, + 140, 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, + 147, 148, 149, 150, 603, 0, 604, 0, 1008, 154, + 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, + 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, + 173, 174, 175, 176, 177, 178, 179, 1009, 1010, 182, + 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, + 190, 191, 192, 193, 194, 0, 0, 195, 196, 767, + 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, + 0, 206, 207, 208, 209, 1012, 211, 212, 213, 214, + 215, 605, 1013, 217, 0, 218, 219, 1014, 221, 0, + 222, 0, 223, 224, 21, 225, 226, 227, 228, 229, + 230, 0, 231, 0, 1015, 1016, 234, 235, 0, 236, + 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 256, 257, 258, 259, 260, 261, 262, 1017, 1018, + 0, 1019, 0, 266, 267, 268, 269, 270, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 278, 279, 280, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 1020, + 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, 307, 1021, 309, + 1022, 311, 312, 313, 314, 1023, 315, 316, 317, 318, + 1024, 607, 320, 1025, 322, 323, 324, 0, 325, 326, + 0, 0, 1026, 328, 329, 0, 0, 330, 331, 332, + 333, 334, 335, 609, 337, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 25, 26, 27, 0, 347, 348, + 610, 350, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 364, 365, 1027, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 1028, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 32, 0, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 1029, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 429, 430, 612, + 432, 0, 433, 434, 37, 435, 436, 437, 438, 439, + 440, 441, 0, 442, 1030, 1031, 0, 0, 445, 446, + 613, 448, 614, 1032, 450, 451, 615, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 39, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 616, 1033, + 0, 472, 473, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 43, 480, 481, 482, 483, 484, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, + 497, 498, 499, 500, 1034, 0, 44, 0, 0, 0, + 0, 1035, 1036, 1037, 0, 0, 0, 0, 1038, 0, + 1039, 3092, 0, 0, 0, 1040, 1041, 1042, 1043, 109, + 995, 632, 996, 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 0, 0, 998, 0, 0, 121, 122, + 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, - 999, 133, 1000, 1001, 0, 136, 137, 138, 139, 140, - 141, 1002, 599, 142, 143, 1003, 1004, 146, 0, 147, - 148, 149, 150, 600, 0, 601, 0, 1005, 154, 155, + 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, 140, + 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, + 148, 149, 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 1006, 1007, 182, 1008, + 174, 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 764, 198, + 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, - 206, 207, 208, 209, 1009, 211, 212, 213, 214, 215, - 602, 1010, 217, 0, 218, 219, 1011, 221, 0, 222, + 206, 207, 208, 209, 1012, 211, 212, 213, 214, 215, + 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, 21, 225, 226, 227, 228, 229, 230, - 0, 231, 0, 1012, 1013, 234, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 255, - 256, 257, 258, 259, 260, 261, 1014, 1015, 0, 1016, - 0, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 277, 278, 279, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 1017, 289, 290, + 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 256, 257, 258, 259, 260, 261, 262, 1017, 1018, 0, + 1019, 0, 266, 267, 268, 269, 270, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 278, 279, 280, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 1018, 308, 1019, 310, - 311, 312, 313, 1020, 314, 315, 316, 317, 1021, 604, - 319, 1022, 321, 322, 323, 0, 324, 325, 0, 0, - 1023, 327, 328, 0, 0, 329, 330, 331, 332, 333, - 606, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 25, 26, 27, 0, 345, 346, 607, 348, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 1024, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 384, 385, - 386, 387, 1025, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 32, 0, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 410, 411, 412, 1026, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 609, 430, 0, 431, - 432, 37, 433, 434, 435, 436, 437, 438, 439, 0, - 1027, 1028, 0, 0, 442, 443, 610, 445, 611, 1029, - 447, 448, 612, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 39, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 613, 1030, 0, 469, 470, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 43, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 301, 302, 303, 304, 305, 306, 307, 1021, 309, 1022, + 311, 312, 313, 314, 1023, 315, 316, 317, 318, 1024, + 607, 320, 1025, 322, 323, 324, 0, 325, 326, 0, + 0, 1026, 328, 329, 0, 0, 330, 331, 332, 333, + 334, 335, 609, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 25, 26, 27, 0, 347, 348, 610, + 350, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 1027, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 1028, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 32, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 612, 432, + 0, 433, 434, 37, 435, 436, 437, 438, 439, 440, + 441, 0, 442, 1030, 1031, 0, 0, 445, 446, 613, + 448, 614, 1032, 450, 451, 615, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 39, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 616, 1033, 0, + 472, 473, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 43, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 1031, 0, 44, 0, 0, 0, 0, 1032, 1033, 1034, - 0, 0, 0, 0, 1035, 0, 1036, 3089, 0, 0, - 0, 1037, 1038, 1039, 1040, 109, 992, 629, 993, 994, - 995, 996, 997, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, - 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, - 0, 998, 0, 0, 121, 122, 123, 0, 124, 125, - 126, 127, 128, 129, 130, 131, 999, 133, 1000, 1001, - 0, 136, 137, 138, 139, 140, 141, 1002, 599, 142, - 143, 1003, 1004, 146, 0, 147, 148, 149, 150, 600, - 0, 601, 0, 1005, 154, 155, 156, 157, 158, 159, - 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, - 0, 169, 170, 171, 172, 173, 174, 175, 176, 177, - 178, 179, 1006, 1007, 182, 1008, 183, 0, 184, 185, - 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, - 0, 0, 195, 196, 764, 198, 199, 0, 200, 201, - 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, - 1009, 211, 212, 213, 214, 215, 602, 1010, 217, 0, - 218, 219, 1011, 221, 0, 222, 0, 223, 224, 21, - 225, 226, 227, 228, 229, 230, 0, 231, 0, 1012, - 1013, 234, 0, 235, 236, 237, 238, 239, 240, 241, - 242, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 255, 256, 257, 258, 259, - 260, 261, 1014, 1015, 0, 1016, 0, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 277, 278, 279, 0, 280, 281, 282, 283, 284, - 285, 286, 287, 1017, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 1018, 308, 1019, 310, 311, 312, 313, 1020, - 314, 315, 316, 317, 1021, 604, 319, 1022, 321, 322, - 323, 0, 324, 325, 0, 0, 1023, 327, 328, 0, - 0, 329, 330, 331, 332, 333, 606, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 25, 26, 27, - 0, 345, 346, 607, 348, 349, 350, 351, 352, 353, - 354, 0, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 1024, 365, 366, 367, 368, 0, 369, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 384, 385, 386, 387, 1025, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 32, 0, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, 410, 411, 412, 1026, 414, 0, 415, 416, - 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 427, 428, 609, 430, 0, 431, 432, 37, 433, 434, - 435, 436, 437, 438, 439, 0, 1027, 1028, 0, 0, - 442, 443, 610, 445, 611, 1029, 447, 448, 612, 450, - 451, 452, 453, 454, 0, 0, 455, 456, 457, 39, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 613, 1030, 0, 469, 470, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 43, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 1031, 0, 44, 0, - 0, 0, 0, 1032, 1033, 1034, 0, 0, 0, 0, - 1035, 0, 1036, 0, 0, 0, 0, 1037, 1038, 1039, - 1040, 1263, 992, 629, 993, 994, 995, 996, 997, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 1264, 118, - 119, 120, 0, 0, 0, 1265, 0, 998, 0, 0, - 1266, 122, 123, 0, 124, 125, 126, 1267, 128, 129, - 130, 131, 999, 1268, 1000, 1001, 0, 136, 137, 138, - 139, 140, 141, 1002, 599, 142, 143, 1003, 1004, 146, - 0, 147, 148, 149, 150, 600, 0, 1269, 0, 1270, - 154, 155, 156, 157, 158, 1271, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 1272, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 1006, 1007, - 182, 1008, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 1273, 0, 195, 196, - 764, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 1009, 211, 212, 213, - 214, 215, 602, 1010, 217, 0, 218, 219, 1011, 221, - 0, 222, 0, 223, 1274, 0, 1275, 226, 227, 1276, - 1277, 230, 0, 231, 0, 1012, 1013, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 1278, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 1279, 256, 257, 258, 259, 260, 261, 1014, 1015, - 0, 1016, 0, 265, 1280, 1281, 268, 1282, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 1283, 278, 1284, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 1285, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 1018, 1286, - 1019, 310, 311, 312, 313, 1020, 314, 315, 1287, 317, - 1021, 604, 319, 1022, 321, 322, 323, 0, 324, 325, - 0, 0, 1023, 327, 328, 0, 0, 329, 330, 1288, - 332, 1289, 606, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 607, - 1290, 349, 350, 351, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 1024, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 1291, 385, 386, 387, 1025, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 1292, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, 1293, 411, - 412, 1026, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 1294, 428, 609, 430, - 0, 431, 432, 0, 433, 1295, 435, 436, 437, 438, - 439, 0, 1027, 1028, 0, 0, 442, 443, 610, 445, - 611, 1029, 447, 448, 1296, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 467, 1030, 1297, 469, - 1298, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 1031, 0, 0, 0, 0, 0, 0, 1032, - 1033, 1034, 0, 0, 0, 0, 1035, 0, 1036, 1299, - 0, 0, 0, 1037, 1038, 1039, 1040, 109, 992, 629, - 993, 994, 995, 996, 997, 0, 0, 0, 0, 0, + 498, 499, 500, 1034, 0, 44, 0, 0, 0, 0, + 1035, 1036, 1037, 0, 0, 0, 0, 1038, 0, 1039, + 0, 0, 0, 0, 1040, 1041, 1042, 1043, 1266, 995, + 632, 996, 997, 998, 999, 1000, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, + 113, 114, 115, 116, 117, 1267, 118, 119, 120, 0, + 0, 0, 1268, 0, 1001, 0, 0, 1269, 122, 123, + 0, 124, 125, 126, 1270, 128, 129, 130, 131, 1002, + 1271, 1003, 1004, 0, 136, 137, 138, 139, 140, 141, + 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, 148, + 149, 150, 603, 0, 1272, 0, 1273, 154, 155, 156, + 157, 158, 1274, 160, 161, 162, 0, 163, 164, 165, + 166, 167, 168, 0, 1275, 170, 171, 172, 173, 174, + 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, + 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, + 192, 193, 194, 1276, 0, 195, 196, 767, 198, 199, + 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, + 207, 208, 209, 1012, 211, 212, 213, 214, 215, 605, + 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, + 223, 1277, 0, 1278, 226, 227, 1279, 1280, 230, 0, + 231, 0, 1015, 1016, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 1281, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 1282, + 257, 258, 259, 260, 261, 262, 1017, 1018, 0, 1019, + 0, 266, 1283, 1284, 269, 1285, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 1286, 279, 1287, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 1288, 290, 291, + 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 1021, 1289, 1022, 311, + 312, 313, 314, 1023, 315, 316, 1290, 318, 1024, 607, + 320, 1025, 322, 323, 324, 0, 325, 326, 0, 0, + 1026, 328, 329, 0, 0, 330, 331, 332, 1291, 334, + 1292, 609, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 610, 1293, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 1027, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 1294, + 387, 388, 389, 1028, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 0, 1295, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 1296, 413, 414, + 1029, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 1297, 430, 612, 432, 0, + 433, 434, 0, 435, 1298, 437, 438, 439, 440, 441, + 0, 442, 1030, 1031, 0, 0, 445, 446, 613, 448, + 614, 1032, 450, 451, 1299, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 470, 1033, 1300, 472, + 1301, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 1034, 0, 0, 0, 0, 0, 0, 1035, + 1036, 1037, 0, 0, 0, 0, 1038, 0, 1039, 1302, + 0, 0, 0, 1040, 1041, 1042, 1043, 109, 995, 632, + 996, 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, - 0, 0, 0, 998, 0, 0, 121, 122, 123, 0, - 124, 125, 126, 127, 128, 129, 130, 131, 999, 133, - 1000, 1001, 1462, 136, 137, 138, 139, 140, 141, 1002, - 599, 142, 143, 1003, 1004, 146, 0, 147, 148, 149, - 150, 600, 0, 601, 0, 1005, 154, 155, 156, 157, + 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, + 124, 125, 126, 127, 128, 129, 130, 131, 1002, 133, + 1003, 1004, 1465, 136, 137, 138, 139, 140, 141, 1005, + 602, 142, 143, 1006, 1007, 146, 0, 147, 148, 149, + 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 1006, 1007, 182, 1008, 183, 0, + 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, - 193, 194, 0, 0, 195, 196, 764, 198, 199, 0, + 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 1009, 211, 212, 213, 214, 215, 602, 1010, - 217, 0, 218, 219, 1011, 221, 0, 222, 0, 223, - 224, 1463, 225, 226, 227, 228, 229, 230, 0, 231, - 0, 1012, 1013, 234, 0, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 255, 256, 257, - 258, 259, 260, 261, 1014, 1015, 0, 1016, 0, 265, + 208, 209, 1012, 211, 212, 213, 214, 215, 605, 1013, + 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, + 224, 1466, 225, 226, 227, 228, 229, 230, 0, 231, + 0, 1015, 1016, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 256, 257, + 258, 259, 260, 261, 262, 1017, 1018, 0, 1019, 0, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, - 0, 1464, 276, 277, 278, 279, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 1017, 289, 290, 291, 292, + 276, 0, 1467, 277, 278, 279, 280, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 1018, 308, 1019, 310, 311, 312, - 313, 1020, 314, 315, 316, 317, 1021, 604, 319, 1022, - 321, 322, 323, 0, 324, 325, 0, 0, 1023, 327, - 328, 0, 0, 329, 330, 331, 332, 333, 606, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 607, 348, 349, 350, 351, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 1024, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 384, 385, 386, 387, - 1025, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 0, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 410, 411, 412, 1026, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 609, 430, 0, 431, 432, 0, - 433, 434, 435, 436, 437, 438, 439, 0, 1027, 1028, - 0, 1465, 442, 443, 610, 445, 611, 1029, 447, 448, - 612, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 467, 1030, 0, 469, 470, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, + 303, 304, 305, 306, 307, 1021, 309, 1022, 311, 312, + 313, 314, 1023, 315, 316, 317, 318, 1024, 607, 320, + 1025, 322, 323, 324, 0, 325, 326, 0, 0, 1026, + 328, 329, 0, 0, 330, 331, 332, 333, 334, 335, + 609, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 610, 350, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 1027, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, + 388, 389, 1028, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 1029, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 612, 432, 0, 433, + 434, 0, 435, 436, 437, 438, 439, 440, 441, 0, + 442, 1030, 1031, 0, 1468, 445, 446, 613, 448, 614, + 1032, 450, 451, 615, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 1033, 0, 472, 473, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 1031, 0, - 0, 0, 0, 0, 0, 1032, 1033, 1034, 0, 0, - 0, 0, 1035, 0, 1036, 0, 0, 0, 0, 1037, - 1038, 1039, 1040, 1263, 992, 629, 993, 994, 995, 996, - 997, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, - 0, 118, 119, 120, 0, 0, 0, 1265, 0, 998, - 0, 0, 1266, 122, 123, 0, 124, 125, 126, 1267, - 128, 129, 130, 131, 999, 1268, 1000, 1001, 0, 136, - 137, 138, 139, 140, 141, 1002, 599, 142, 143, 1003, - 1004, 146, 0, 147, 148, 149, 150, 600, 0, 1269, - 0, 1270, 154, 155, 156, 157, 158, 1271, 160, 161, - 162, 0, 163, 164, 165, 166, 167, 168, 0, 1272, - 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, - 1006, 1007, 182, 1008, 183, 0, 184, 185, 186, 187, - 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, - 195, 196, 764, 198, 199, 0, 200, 201, 202, 0, - 203, 204, 205, 0, 206, 207, 208, 209, 1009, 211, - 212, 213, 214, 215, 602, 1010, 217, 0, 218, 219, - 1011, 221, 0, 222, 0, 223, 1274, 0, 1275, 226, - 227, 1276, 1277, 230, 0, 231, 0, 1012, 1013, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 1278, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 1279, 256, 257, 258, 259, 260, 261, - 1014, 1015, 0, 1016, 0, 265, 1280, 1281, 268, 1282, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 1283, - 278, 1284, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 1285, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 1018, 1286, 1019, 310, 311, 312, 313, 1020, 314, 315, - 1287, 317, 1021, 604, 319, 1022, 321, 322, 323, 0, - 324, 325, 0, 0, 1023, 327, 328, 0, 0, 329, - 330, 1288, 332, 1289, 606, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 607, 1290, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 1024, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 1291, 385, 386, 387, 1025, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 1292, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 1293, 411, 412, 1026, 414, 0, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 1294, 428, - 609, 430, 0, 431, 432, 0, 433, 1295, 435, 436, - 437, 438, 439, 0, 1027, 1028, 0, 0, 442, 443, - 610, 445, 611, 1029, 447, 448, 1296, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 1030, - 2207, 469, 1298, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 1031, 0, 0, 0, 0, 0, - 0, 1032, 1033, 1034, 0, 0, 0, 0, 1035, 0, - 1036, 0, 0, 0, 0, 1037, 1038, 1039, 1040, 1263, - 992, 629, 993, 994, 995, 996, 997, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, - 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 1265, 0, 998, 0, 0, 1266, 122, - 123, 0, 124, 125, 126, 1267, 128, 129, 130, 131, - 999, 1268, 1000, 1001, 0, 136, 137, 138, 139, 140, - 141, 1002, 599, 142, 143, 1003, 1004, 146, 0, 147, - 148, 149, 150, 600, 0, 1269, 0, 1270, 154, 155, - 156, 157, 158, 1271, 160, 161, 162, 0, 163, 164, - 165, 166, 167, 168, 0, 1272, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 1006, 1007, 182, 1008, - 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 764, 198, - 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, - 206, 207, 208, 209, 1009, 211, 212, 213, 214, 215, - 602, 1010, 217, 0, 218, 219, 1011, 221, 0, 222, - 0, 223, 1274, 0, 1275, 226, 227, 1276, 1277, 230, - 0, 231, 0, 1012, 1013, 234, 0, 235, 236, 237, - 238, 239, 240, 241, 1278, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 1279, - 256, 257, 258, 259, 260, 261, 1014, 1015, 0, 1016, - 0, 265, 1280, 1281, 268, 1282, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 1283, 278, 1284, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 1285, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 1018, 1286, 1019, 310, - 311, 312, 313, 1020, 314, 315, 1287, 317, 1021, 604, - 319, 1022, 321, 322, 323, 0, 324, 325, 0, 0, - 1023, 327, 328, 0, 0, 329, 330, 1288, 332, 1289, - 606, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 607, 1290, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 1024, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 1291, 385, - 386, 387, 1025, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 0, 1292, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 1293, 411, 412, 1026, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 1294, 428, 609, 430, 0, 431, - 432, 0, 433, 1295, 435, 436, 437, 438, 439, 0, - 1027, 1028, 0, 0, 442, 443, 610, 445, 611, 1029, - 447, 448, 1296, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 467, 1030, 0, 469, 1298, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 1031, 0, 0, 0, 0, 0, 0, 1032, 1033, 1034, - 0, 0, 0, 0, 1035, 0, 1036, 2254, 0, 0, - 0, 1037, 1038, 1039, 1040, 109, 992, 629, 993, 994, - 995, 996, 997, 0, 0, 0, 0, 0, 0, 0, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 1034, 0, 0, 0, 0, 0, 0, 1035, 1036, + 1037, 0, 0, 0, 0, 1038, 0, 1039, 0, 0, + 0, 0, 1040, 1041, 1042, 1043, 1266, 995, 632, 996, + 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, + 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, + 1268, 0, 1001, 0, 0, 1269, 122, 123, 0, 124, + 125, 126, 1270, 128, 129, 130, 131, 1002, 1271, 1003, + 1004, 0, 136, 137, 138, 139, 140, 141, 1005, 602, + 142, 143, 1006, 1007, 146, 0, 147, 148, 149, 150, + 603, 0, 1272, 0, 1273, 154, 155, 156, 157, 158, + 1274, 160, 161, 162, 0, 163, 164, 165, 166, 167, + 168, 0, 1275, 170, 171, 172, 173, 174, 175, 176, + 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, + 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, + 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, + 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, + 209, 1012, 211, 212, 213, 214, 215, 605, 1013, 217, + 0, 218, 219, 1014, 221, 0, 222, 0, 223, 1277, + 0, 1278, 226, 227, 1279, 1280, 230, 0, 231, 0, + 1015, 1016, 234, 235, 0, 236, 237, 238, 239, 240, + 241, 242, 1281, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 1282, 257, 258, + 259, 260, 261, 262, 1017, 1018, 0, 1019, 0, 266, + 1283, 1284, 269, 1285, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 1286, 279, 1287, 0, 281, 282, 283, + 284, 285, 286, 287, 288, 1288, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 1021, 1289, 1022, 311, 312, 313, + 314, 1023, 315, 316, 1290, 318, 1024, 607, 320, 1025, + 322, 323, 324, 0, 325, 326, 0, 0, 1026, 328, + 329, 0, 0, 330, 331, 332, 1291, 334, 1292, 609, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 610, 1293, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 1027, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 1294, 387, 388, + 389, 1028, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 0, 1295, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 1296, 413, 414, 1029, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 1297, 430, 612, 432, 0, 433, 434, + 0, 435, 1298, 437, 438, 439, 440, 441, 0, 442, + 1030, 1031, 0, 0, 445, 446, 613, 448, 614, 1032, + 450, 451, 1299, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 1033, 2210, 472, 1301, 474, + 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 1034, 0, 0, 0, 0, 0, 0, 1035, 1036, 1037, + 0, 0, 0, 0, 1038, 0, 1039, 0, 0, 0, + 0, 1040, 1041, 1042, 1043, 1266, 995, 632, 996, 997, + 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, - 116, 117, -1064, 118, 119, 120, 0, 0, 0, 0, - -1064, 998, 0, 0, 121, 122, 123, 0, 124, 125, - 126, 127, 128, 129, 130, 131, 999, 133, 1000, 1001, - 0, 136, 137, 138, 139, 140, 141, 1002, 599, 142, - 143, 1003, 1004, 146, 0, 147, 148, 149, 150, 600, - 0, 601, 0, 1005, 154, 155, 156, 157, 158, 159, + 116, 117, 0, 118, 119, 120, 0, 0, 0, 1268, + 0, 1001, 0, 0, 1269, 122, 123, 0, 124, 125, + 126, 1270, 128, 129, 130, 131, 1002, 1271, 1003, 1004, + 0, 136, 137, 138, 139, 140, 141, 1005, 602, 142, + 143, 1006, 1007, 146, 0, 147, 148, 149, 150, 603, + 0, 1272, 0, 1273, 154, 155, 156, 157, 158, 1274, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, - 0, 169, 170, 171, 172, 173, 174, 175, 176, 177, - 178, 179, 1006, 1007, 182, 1008, 183, 0, 184, 185, + 0, 1275, 170, 171, 172, 173, 174, 175, 176, 177, + 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, - 0, 0, 195, 196, 764, 198, 199, 0, 200, 201, + 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, - 1009, 211, 212, 213, 214, 215, 602, 1010, 217, 0, - 218, 219, 1011, 221, 0, 222, 0, 223, 224, 0, - 225, 226, 227, 228, 229, 230, 0, 231, 0, 1012, - 1013, 234, 0, 235, 236, 237, 238, 239, 240, 241, - 242, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 255, 256, 257, 258, 259, - 260, 261, 1014, 1015, 0, 1016, 0, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 277, 278, 279, 0, 280, 281, 282, 283, 284, - 285, 286, 287, 1017, 289, 290, 291, 292, 293, 294, + 1012, 211, 212, 213, 214, 215, 605, 1013, 217, 0, + 218, 219, 1014, 221, 0, 222, 0, 223, 1277, 0, + 1278, 226, 227, 1279, 1280, 230, 0, 231, 0, 1015, + 1016, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 1281, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 1282, 257, 258, 259, + 260, 261, 262, 1017, 1018, 0, 1019, 0, 266, 1283, + 1284, 269, 1285, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 1286, 279, 1287, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 1288, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 1018, 308, 1019, 310, 311, 312, 313, 1020, - 314, 315, 316, 317, 1021, 604, 319, 1022, 321, 322, - 323, 0, 324, 325, 0, 0, 1023, 327, 328, 0, - 0, 329, 330, 331, 332, 333, 606, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 0, 0, 0, - 0, 345, 346, 607, 348, 349, 350, 351, 352, 353, - 354, 0, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 1024, 365, 366, 367, 368, 0, 369, 370, + 305, 306, 307, 1021, 1289, 1022, 311, 312, 313, 314, + 1023, 315, 316, 1290, 318, 1024, 607, 320, 1025, 322, + 323, 324, 0, 325, 326, 0, 0, 1026, 328, 329, + 0, 0, 330, 331, 332, 1291, 334, 1292, 609, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 610, 1293, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 1027, 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 384, 385, 386, 387, 1025, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 0, 0, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, 410, 411, 412, 1026, 414, -1064, 415, 416, + 381, 382, 383, 0, 384, 385, 1294, 387, 388, 389, + 1028, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 0, 1295, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 1296, 413, 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 427, 428, 609, 430, 0, 431, 432, 0, 433, 434, - 435, 436, 437, 438, 439, 0, 1027, 1028, 0, 0, - 442, 443, 610, 445, 611, 1029, 447, 448, 612, 450, - 451, 452, 453, 454, 0, 0, 455, 456, 457, 0, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 467, 1030, 0, 469, 470, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 0, 477, 478, 479, 480, 481, + 427, 428, 1297, 430, 612, 432, 0, 433, 434, 0, + 435, 1298, 437, 438, 439, 440, 441, 0, 442, 1030, + 1031, 0, 0, 445, 446, 613, 448, 614, 1032, 450, + 451, 1299, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 1033, 0, 472, 1301, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 1031, 0, 0, 0, - 0, 0, 0, 1032, 1033, 1034, 0, 0, 0, 0, - 1035, 0, 1036, 0, 0, 0, 0, 1037, 1038, 1039, - 1040, 1263, 992, 629, 993, 994, 995, 996, 997, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, - 119, 120, 0, 0, 0, 1265, 0, 998, 0, 0, - 1266, 122, 123, 0, 124, 125, 126, 1267, 128, 129, - 130, 131, 999, 1268, 1000, 1001, 0, 136, 137, 138, - 139, 140, 141, 1002, 599, 142, 143, 1003, 1004, 146, - 0, 147, 148, 149, 150, 600, 0, 1269, 0, 1270, - 154, 155, 156, 157, 158, 1271, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 1272, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 1006, 1007, - 182, 1008, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, - 764, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 1009, 211, 212, 213, - 214, 215, 602, 1010, 217, 0, 218, 219, 1011, 221, - 0, 222, 0, 223, 1274, 0, 1275, 226, 227, 1276, - 1277, 230, 0, 231, 0, 1012, 1013, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 1278, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 1279, 256, 257, 258, 259, 260, 261, 1014, 1015, - 0, 1016, 0, 265, 1280, 1281, 268, 1282, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 1283, 278, 1284, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 1285, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 1018, 1286, - 1019, 310, 311, 312, 313, 1020, 314, 315, 1287, 317, - 1021, 604, 319, 1022, 321, 322, 323, 0, 324, 325, - 0, 0, 1023, 327, 328, 0, 0, 329, 330, 1288, - 332, 1289, 606, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 607, - 1290, 349, 350, 351, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 1024, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 1291, 385, 386, 387, 1025, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 1292, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, 1293, 411, - 412, 1026, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 1294, 428, 609, 430, - 0, 431, 432, 0, 433, 1295, 435, 436, 437, 438, - 439, 0, 1027, 1028, 0, 0, 442, 443, 610, 445, - 611, 1029, 447, 448, 1296, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 467, 1030, 0, 469, - 1298, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 1031, 0, 0, 0, 0, 0, 0, 1032, - 1033, 1034, 0, 0, 0, 0, 1035, 0, 1036, 2960, - 0, 0, 0, 1037, 1038, 1039, 1040, 1263, 992, 629, - 993, 994, 995, 996, 997, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, - 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, - 0, 1265, 0, 998, 0, 0, 1266, 122, 123, 0, - 124, 125, 126, 1267, 128, 129, 130, 131, 999, 1268, - 1000, 1001, 0, 136, 137, 138, 139, 140, 141, 1002, - 599, 142, 143, 1003, 1004, 146, 0, 147, 148, 149, - 150, 600, 0, 1269, 0, 1270, 154, 155, 156, 157, - 158, 1271, 160, 161, 162, 0, 163, 164, 165, 166, - 167, 168, 0, 1272, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 1006, 1007, 182, 1008, 183, 0, - 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, - 193, 194, 0, 0, 195, 196, 764, 198, 199, 0, - 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 1009, 211, 212, 213, 214, 215, 602, 1010, - 217, 0, 218, 219, 1011, 221, 0, 222, 0, 223, - 1274, 0, 1275, 226, 227, 1276, 1277, 230, 0, 231, - 0, 1012, 1013, 234, 0, 235, 236, 237, 238, 239, - 240, 241, 1278, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 1279, 256, 257, - 258, 259, 260, 261, 1014, 1015, 0, 1016, 0, 265, - 1280, 1281, 268, 1282, 270, 271, 272, 273, 274, 275, - 0, 0, 276, 1283, 278, 1284, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 1285, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 1018, 1286, 1019, 310, 311, 312, - 313, 1020, 314, 315, 1287, 317, 1021, 604, 319, 1022, - 321, 322, 323, 0, 324, 325, 0, 0, 1023, 327, - 328, 0, 0, 329, 330, 1288, 332, 1289, 606, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 607, 1290, 349, 350, 351, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 1024, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 1291, 385, 386, 387, - 1025, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 1292, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 1293, 411, 412, 1026, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 1294, 428, 609, 430, 0, 431, 432, 0, - 433, 1295, 435, 436, 437, 438, 439, 0, 1027, 1028, - 0, 0, 442, 443, 610, 445, 611, 1029, 447, 448, - 1296, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 467, 1030, 0, 469, 1298, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 1031, 0, - 0, 0, 0, 0, 0, 1032, 1033, 1034, 0, 0, - 0, 0, 1035, 0, 1036, 0, 0, 0, 0, 1037, - 1038, 1039, 1040, 109, 992, 629, 993, 994, 0, 996, - 997, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 1034, + 0, 0, 0, 0, 0, 0, 1035, 1036, 1037, 0, + 0, 0, 0, 1038, 0, 1039, 2257, 0, 0, 0, + 1040, 1041, 1042, 1043, 109, 995, 632, 996, 997, 998, + 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, + 117, -1070, 118, 119, 120, 0, 0, 0, 0, -1070, + 1001, 0, 0, 121, 122, 123, 0, 124, 125, 126, + 127, 128, 129, 130, 131, 1002, 133, 1003, 1004, 0, + 136, 137, 138, 139, 140, 141, 1005, 602, 142, 143, + 1006, 1007, 146, 0, 147, 148, 149, 150, 603, 0, + 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, + 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, + 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, + 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, + 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, + 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 1012, + 211, 212, 213, 214, 215, 605, 1013, 217, 0, 218, + 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, + 226, 227, 228, 229, 230, 0, 231, 0, 1015, 1016, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 256, 257, 258, 259, 260, + 261, 262, 1017, 1018, 0, 1019, 0, 266, 267, 268, + 269, 270, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 278, 279, 280, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 1021, 309, 1022, 311, 312, 313, 314, 1023, + 315, 316, 317, 318, 1024, 607, 320, 1025, 322, 323, + 324, 0, 325, 326, 0, 0, 1026, 328, 329, 0, + 0, 330, 331, 332, 333, 334, 335, 609, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 610, 350, 351, 352, 353, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 1027, 367, 368, 369, 370, 0, 371, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 386, 387, 388, 389, 1028, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 1029, 416, -1070, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, + 428, 429, 430, 612, 432, 0, 433, 434, 0, 435, + 436, 437, 438, 439, 440, 441, 0, 442, 1030, 1031, + 0, 0, 445, 446, 613, 448, 614, 1032, 450, 451, + 615, 453, 454, 455, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 470, 1033, 0, 472, 473, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 498, 499, 500, 1034, 0, + 0, 0, 0, 0, 0, 1035, 1036, 1037, 0, 0, + 0, 0, 1038, 0, 1039, 0, 0, 0, 0, 1040, + 1041, 1042, 1043, 1266, 995, 632, 996, 997, 998, 999, + 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, - 0, 118, 119, 120, 0, 0, 0, 0, 0, 998, - 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, - 128, 129, 130, 131, 999, 133, 1000, 1001, 0, 136, - 137, 138, 139, 140, 141, 1002, 599, 142, 143, 1003, - 1004, 146, 0, 147, 148, 149, 150, 600, 0, 601, - 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, - 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, + 0, 118, 119, 120, 0, 0, 0, 1268, 0, 1001, + 0, 0, 1269, 122, 123, 0, 124, 125, 126, 1270, + 128, 129, 130, 131, 1002, 1271, 1003, 1004, 0, 136, + 137, 138, 139, 140, 141, 1005, 602, 142, 143, 1006, + 1007, 146, 0, 147, 148, 149, 150, 603, 0, 1272, + 0, 1273, 154, 155, 156, 157, 158, 1274, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 1275, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, - 1006, 1007, 182, 0, 183, 0, 184, 185, 186, 187, + 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, - 195, 196, 764, 198, 199, 0, 200, 201, 202, 0, - 203, 204, 205, 0, 206, 207, 208, 209, 1009, 211, - 212, 213, 214, 215, 602, 1010, 217, 0, 218, 219, - 1011, 221, 0, 222, 0, 223, 224, 21, 225, 226, - 227, 228, 229, 230, 0, 231, 0, 1012, 1013, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 255, 256, 257, 258, 259, 260, 261, - 1014, 1015, 0, 1016, 0, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 277, - 278, 279, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 1017, 289, 290, 291, 292, 293, 294, 295, 296, + 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, + 203, 204, 205, 0, 206, 207, 208, 209, 1012, 211, + 212, 213, 214, 215, 605, 1013, 217, 0, 218, 219, + 1014, 221, 0, 222, 0, 223, 1277, 0, 1278, 226, + 227, 1279, 1280, 230, 0, 231, 0, 1015, 1016, 234, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 1281, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 1282, 257, 258, 259, 260, 261, + 262, 1017, 1018, 0, 1019, 0, 266, 1283, 1284, 269, + 1285, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 1286, 279, 1287, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 1288, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 1018, 308, 1019, 310, 311, 312, 313, 0, 314, 315, - 316, 317, 1021, 604, 319, 1022, 321, 322, 323, 0, - 324, 325, 0, 0, 326, 327, 328, 0, 0, 329, - 330, 331, 332, 333, 606, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 25, 26, 27, 0, 345, - 346, 607, 348, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 1024, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 384, 385, 386, 387, 1025, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 32, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 410, 411, 412, 1026, 414, 0, 415, 416, 417, 418, + 307, 1021, 1289, 1022, 311, 312, 313, 314, 1023, 315, + 316, 1290, 318, 1024, 607, 320, 1025, 322, 323, 324, + 0, 325, 326, 0, 0, 1026, 328, 329, 0, 0, + 330, 331, 332, 1291, 334, 1292, 609, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 610, 1293, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 1027, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 1294, 387, 388, 389, 1028, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 1295, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 1296, 413, 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 609, 430, 0, 431, 432, 37, 433, 434, 435, 436, - 437, 438, 439, 0, 1027, 1028, 0, 0, 442, 443, - 610, 445, 611, 1029, 447, 448, 612, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 39, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 613, 1030, - 0, 469, 470, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 43, 477, 478, 479, 480, 481, 482, 483, + 1297, 430, 612, 432, 0, 433, 434, 0, 435, 1298, + 437, 438, 439, 440, 441, 0, 442, 1030, 1031, 0, + 0, 445, 446, 613, 448, 614, 1032, 450, 451, 1299, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 1033, 0, 472, 1301, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 0, 0, 44, 0, 0, 0, - 109, 992, 629, 993, 994, 995, 996, 997, 1035, 0, - 1036, 0, 0, 0, 0, 1037, 1038, 1039, 1040, 110, - 111, 112, 113, 114, 115, 116, 117, 1640, 118, 119, - 120, 0, 0, 0, 0, 0, 998, 0, 0, 121, + 494, 495, 496, 497, 498, 499, 500, 1034, 0, 0, + 0, 0, 0, 0, 1035, 1036, 1037, 0, 0, 0, + 0, 1038, 0, 1039, 2963, 0, 0, 0, 1040, 1041, + 1042, 1043, 1266, 995, 632, 996, 997, 998, 999, 1000, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, + 118, 119, 120, 0, 0, 0, 1268, 0, 1001, 0, + 0, 1269, 122, 123, 0, 124, 125, 126, 1270, 128, + 129, 130, 131, 1002, 1271, 1003, 1004, 0, 136, 137, + 138, 139, 140, 141, 1005, 602, 142, 143, 1006, 1007, + 146, 0, 147, 148, 149, 150, 603, 0, 1272, 0, + 1273, 154, 155, 156, 157, 158, 1274, 160, 161, 162, + 0, 163, 164, 165, 166, 167, 168, 0, 1275, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 1009, + 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, + 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, + 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, + 204, 205, 0, 206, 207, 208, 209, 1012, 211, 212, + 213, 214, 215, 605, 1013, 217, 0, 218, 219, 1014, + 221, 0, 222, 0, 223, 1277, 0, 1278, 226, 227, + 1279, 1280, 230, 0, 231, 0, 1015, 1016, 234, 235, + 0, 236, 237, 238, 239, 240, 241, 242, 1281, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 1282, 257, 258, 259, 260, 261, 262, + 1017, 1018, 0, 1019, 0, 266, 1283, 1284, 269, 1285, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 1286, + 279, 1287, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 1288, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 1021, 1289, 1022, 311, 312, 313, 314, 1023, 315, 316, + 1290, 318, 1024, 607, 320, 1025, 322, 323, 324, 0, + 325, 326, 0, 0, 1026, 328, 329, 0, 0, 330, + 331, 332, 1291, 334, 1292, 609, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 610, 1293, 351, 352, 353, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 1027, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 1294, 387, 388, 389, 1028, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 1295, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 1296, 413, 414, 1029, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 1297, + 430, 612, 432, 0, 433, 434, 0, 435, 1298, 437, + 438, 439, 440, 441, 0, 442, 1030, 1031, 0, 0, + 445, 446, 613, 448, 614, 1032, 450, 451, 1299, 453, + 454, 455, 456, 457, 0, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 470, 1033, 0, 472, 1301, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, 499, 500, 1034, 0, 0, 0, + 0, 0, 0, 1035, 1036, 1037, 0, 0, 0, 0, + 1038, 0, 1039, 0, 0, 0, 0, 1040, 1041, 1042, + 1043, 109, 995, 632, 996, 997, 0, 999, 1000, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, + 119, 120, 0, 0, 0, 0, 0, 1001, 0, 0, + 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, + 130, 131, 1002, 133, 1003, 1004, 0, 136, 137, 138, + 139, 140, 141, 1005, 602, 142, 143, 1006, 1007, 146, + 0, 147, 148, 149, 150, 603, 0, 604, 0, 153, + 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, + 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 1009, 1010, + 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 1012, 211, 212, 213, + 214, 215, 605, 1013, 217, 0, 218, 219, 1014, 221, + 0, 222, 0, 223, 224, 21, 225, 226, 227, 228, + 229, 230, 0, 231, 0, 1015, 1016, 234, 235, 0, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 256, 257, 258, 259, 260, 261, 262, 1017, + 1018, 0, 1019, 0, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 278, 279, + 280, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 1021, + 309, 1022, 311, 312, 313, 314, 0, 315, 316, 317, + 318, 1024, 607, 320, 1025, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 332, 333, 334, 335, 609, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 25, 26, 27, 0, 347, + 348, 610, 350, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 1027, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 1028, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 32, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 1029, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 612, 432, 0, 433, 434, 37, 435, 436, 437, 438, + 439, 440, 441, 0, 442, 1030, 1031, 0, 0, 445, + 446, 613, 448, 614, 1032, 450, 451, 615, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 39, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 616, + 1033, 0, 472, 473, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 43, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 0, 0, 44, 0, 0, + 0, 109, 995, 632, 996, 997, 998, 999, 1000, 1038, + 0, 1039, 0, 0, 0, 0, 1040, 1041, 1042, 1043, + 110, 111, 112, 113, 114, 115, 116, 117, 1643, 118, + 119, 120, 0, 0, 0, 0, 0, 1001, 0, 0, + 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, + 130, 131, 1002, 133, 1003, 1004, 0, 136, 137, 138, + 139, 140, 141, 1005, 602, 142, 143, 1006, 1007, 146, + 0, 147, 148, 149, 150, 603, 0, 604, 0, 1008, + 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, + 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 1009, 1010, + 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 1012, 211, 212, 213, + 214, 215, 605, 1013, 217, 0, 218, 219, 1014, 221, + 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, + 229, 230, 0, 231, 0, 1015, 1016, 234, 235, 0, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 256, 257, 258, 259, 260, 261, 262, 1017, + 1018, 0, 1019, 0, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 278, 279, + 280, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 1021, + 309, 1022, 311, 312, 313, 314, 1023, 315, 316, 317, + 318, 1024, 607, 320, 1025, 322, 323, 324, 0, 325, + 326, 0, 0, 1026, 328, 329, 0, 0, 330, 331, + 332, 333, 334, 335, 609, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 610, 350, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 1027, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 1028, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 1029, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 612, 432, 0, 433, 434, 0, 435, 436, 437, 438, + 439, 440, 441, 0, 442, 1030, 1031, 0, 0, 445, + 446, 613, 448, 614, 1032, 450, 451, 615, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 1033, 0, 472, 473, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 1034, 0, 0, 0, 0, + 0, 0, 1035, 1036, 1037, 0, 0, 0, 0, 1038, + 0, 1039, 0, 0, 0, 0, 1040, 1041, 1042, 1043, + 109, 1658, 632, 996, 997, 998, 1659, 1000, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, + 111, 112, 113, 114, 115, 116, 117, 1660, 118, 119, + 120, 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, - 131, 999, 133, 1000, 1001, 0, 136, 137, 138, 139, - 140, 141, 1002, 599, 142, 143, 1003, 1004, 146, 0, - 147, 148, 149, 150, 600, 0, 601, 0, 1005, 154, + 131, 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, + 140, 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, + 147, 148, 149, 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, - 173, 174, 175, 176, 177, 178, 179, 1006, 1007, 182, - 1008, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 764, + 173, 174, 175, 176, 177, 178, 179, 1009, 1010, 182, + 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, + 190, 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 1009, 211, 212, 213, 214, - 215, 602, 1010, 217, 0, 218, 219, 1011, 221, 0, + 0, 206, 207, 208, 209, 1012, 211, 212, 213, 214, + 215, 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, - 230, 0, 231, 0, 1012, 1013, 234, 0, 235, 236, + 230, 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 255, 256, 257, 258, 259, 260, 261, 1014, 1015, 0, - 1016, 0, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 277, 278, 279, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 1017, 289, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 256, 257, 258, 259, 260, 261, 262, 1017, 1018, + 0, 1019, 0, 266, 267, 268, 269, 270, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 278, 279, 280, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 1018, 308, 1019, - 310, 311, 312, 313, 1020, 314, 315, 316, 317, 1021, - 604, 319, 1022, 321, 322, 323, 0, 324, 325, 0, - 0, 1023, 327, 328, 0, 0, 329, 330, 331, 332, - 333, 606, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 607, 348, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 1024, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 384, - 385, 386, 387, 1025, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, + 300, 301, 302, 303, 304, 305, 306, 307, 1021, 309, + 1022, 311, 312, 313, 314, 1023, 315, 316, 317, 318, + 1024, 607, 320, 1025, 322, 323, 324, 0, 325, 326, + 0, 0, 1026, 328, 329, 0, 0, 330, 331, 332, + 333, 334, 335, 609, 337, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 0, 0, 0, 0, 347, 348, + 610, 350, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 364, 365, 1027, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 1028, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, - 1026, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 609, 430, 0, - 431, 432, 0, 433, 434, 435, 436, 437, 438, 439, - 0, 1027, 1028, 0, 0, 442, 443, 610, 445, 611, - 1029, 447, 448, 612, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 1030, 0, 469, 470, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 413, 414, 1029, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 429, 430, 612, + 432, 0, 433, 434, 0, 435, 436, 437, 438, 439, + 440, 441, 0, 442, 1030, 1031, 0, 0, 445, 446, + 613, 448, 614, 1032, 450, 451, 615, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 0, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 470, 1033, + 0, 472, 473, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 1031, 0, 0, 0, 0, 0, 0, 1032, 1033, - 1034, 0, 0, 0, 0, 1035, 0, 1036, 0, 0, - 0, 0, 1037, 1038, 1039, 1040, 109, 1655, 629, 993, - 994, 995, 1656, 997, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 1657, 118, 119, 120, 0, 0, 0, - 0, 0, 998, 0, 0, 121, 122, 123, 0, 124, - 125, 126, 127, 128, 129, 130, 131, 999, 133, 1000, - 1001, 0, 136, 137, 138, 139, 140, 141, 1002, 599, - 142, 143, 1003, 1004, 146, 0, 147, 148, 149, 150, - 600, 0, 601, 0, 1005, 154, 155, 156, 157, 158, - 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, - 177, 178, 179, 1006, 1007, 182, 1008, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 764, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 1009, 211, 212, 213, 214, 215, 602, 1010, 217, - 0, 218, 219, 1011, 221, 0, 222, 0, 223, 224, - 0, 225, 226, 227, 228, 229, 230, 0, 231, 0, - 1012, 1013, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 255, 256, 257, 258, - 259, 260, 261, 1014, 1015, 0, 1016, 0, 265, 266, - 267, 268, 269, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 277, 278, 279, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 1017, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 1018, 308, 1019, 310, 311, 312, 313, - 1020, 314, 315, 316, 317, 1021, 604, 319, 1022, 321, - 322, 323, 0, 324, 325, 0, 0, 1023, 327, 328, - 0, 0, 329, 330, 331, 332, 333, 606, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 607, 348, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 1024, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 384, 385, 386, 387, 1025, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 410, 411, 412, 1026, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 609, 430, 0, 431, 432, 0, 433, - 434, 435, 436, 437, 438, 439, 0, 1027, 1028, 0, - 0, 442, 443, 610, 445, 611, 1029, 447, 448, 612, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 1030, 0, 469, 470, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 1031, 0, 0, - 0, 0, 0, 0, 1032, 1033, 1034, 0, 0, 0, - 0, 1035, 0, 1036, 0, 0, 0, 0, 1037, 1038, - 1039, 1040, 109, 992, 629, 993, 994, 995, 996, 997, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 998, 0, - 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, - 129, 130, 131, 999, 133, 1000, 1001, 0, 136, 137, - 138, 139, 140, 141, 1002, 599, 142, 143, 1003, 1004, - 146, 0, 147, 148, 149, 150, 600, 0, 601, 0, - 1005, 154, 155, 156, 157, 158, 159, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 1006, - 1007, 182, 1008, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 764, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 1009, 211, 212, - 213, 214, 215, 602, 1010, 217, 0, 218, 219, 1011, - 221, 0, 222, 0, 223, 224, 1463, 225, 226, 227, - 228, 229, 230, 0, 231, 0, 1012, 1013, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 255, 256, 257, 258, 259, 260, 261, 1014, - 1015, 0, 1016, 0, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 277, 278, - 279, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 1017, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 1018, - 308, 1019, 310, 311, 312, 313, 1020, 314, 315, 316, - 317, 1021, 604, 319, 1022, 321, 322, 323, 0, 324, - 325, 0, 0, 1023, 327, 328, 0, 0, 329, 330, - 331, 332, 333, 606, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 607, 348, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 1024, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 384, 385, 386, 387, 1025, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, - 411, 412, 1026, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 609, - 430, 0, 431, 432, 0, 433, 434, 435, 436, 437, - 438, 439, 0, 1027, 1028, 0, 0, 442, 443, 610, - 445, 611, 1029, 447, 448, 612, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 1030, 0, - 469, 470, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 1031, 0, 0, 0, 0, 0, 0, - 1032, 1033, 1034, 0, 0, 0, 0, 1035, 0, 1036, - 0, 0, 0, 0, 1037, 1038, 1039, 1040, 109, 992, - 629, 993, 994, 995, 996, 997, 0, 0, 0, 0, + 497, 498, 499, 500, 1034, 0, 0, 0, 0, 0, + 0, 1035, 1036, 1037, 0, 0, 0, 0, 1038, 0, + 1039, 0, 0, 0, 0, 1040, 1041, 1042, 1043, 109, + 995, 632, 996, 997, 998, 999, 1000, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, + 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, + 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, 140, + 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, + 148, 149, 150, 603, 0, 604, 0, 1008, 154, 155, + 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 1012, 211, 212, 213, 214, 215, + 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, + 0, 223, 224, 1466, 225, 226, 227, 228, 229, 230, + 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 256, 257, 258, 259, 260, 261, 262, 1017, 1018, 0, + 1019, 0, 266, 267, 268, 269, 270, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 278, 279, 280, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 1021, 309, 1022, + 311, 312, 313, 314, 1023, 315, 316, 317, 318, 1024, + 607, 320, 1025, 322, 323, 324, 0, 325, 326, 0, + 0, 1026, 328, 329, 0, 0, 330, 331, 332, 333, + 334, 335, 609, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 610, + 350, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 1027, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 1028, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 612, 432, + 0, 433, 434, 0, 435, 436, 437, 438, 439, 440, + 441, 0, 442, 1030, 1031, 0, 0, 445, 446, 613, + 448, 614, 1032, 450, 451, 615, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 1033, 0, + 472, 473, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 1034, 0, 0, 0, 0, 0, 0, + 1035, 1036, 1037, 0, 0, 0, 0, 1038, 0, 1039, + 0, 0, 0, 0, 1040, 1041, 1042, 1043, 109, 995, + 632, 996, 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 998, 0, 0, 121, 122, 123, - 0, 124, 125, 126, 127, 128, 129, 130, 131, 999, - 133, 1000, 1001, 0, 136, 137, 138, 139, 140, 141, - 1002, 599, 142, 143, 1003, 1004, 146, 0, 147, 148, - 149, 150, 600, 0, 601, 0, 1005, 154, 155, 156, + 0, 0, 0, 0, 1001, 0, 0, 121, 122, 123, + 0, 124, 125, 126, 127, 128, 129, 130, 131, 1002, + 133, 1003, 1004, 0, 136, 137, 138, 139, 140, 141, + 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, 148, + 149, 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 1006, 1007, 182, 1008, 183, + 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 764, 198, 199, + 192, 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 1009, 211, 212, 213, 214, 215, 602, - 1010, 217, 0, 218, 219, 1011, 221, 0, 222, 0, + 207, 208, 209, 1012, 211, 212, 213, 214, 215, 605, + 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, 0, - 231, 0, 1012, 1013, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 255, 256, - 257, 258, 259, 260, 261, 1014, 1015, 0, 1016, 0, - 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 277, 278, 279, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 1017, 289, 290, 291, + 231, 0, 1015, 1016, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 243, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 256, + 257, 258, 259, 260, 261, 262, 1017, 1018, 0, 1019, + 0, 266, 267, 268, 269, 270, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 278, 279, 280, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 1018, 308, 1019, 310, 311, - 312, 313, 1020, 314, 315, 316, 317, 1021, 604, 319, - 1022, 321, 322, 323, 0, 324, 325, 0, 0, 1023, - 327, 328, 0, 0, 329, 330, 331, 332, 333, 606, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 607, 348, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 1024, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 384, 385, 386, - 387, 1025, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 410, 411, 412, 1026, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 609, 430, 0, 431, 432, - 0, 433, 434, 435, 436, 437, 438, 439, 0, 1027, - 1028, 0, 0, 442, 443, 610, 445, 611, 1029, 447, - 448, 612, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 1030, 0, 469, 470, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 1031, - 0, 0, 0, 0, 0, 0, 1032, 1033, 1034, 0, - 0, 0, 0, 1035, 0, 1036, 1987, 0, 0, 0, - 1037, 1038, 1039, 1040, 109, 992, 629, 993, 994, 995, - 996, 997, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, - 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 998, 0, 0, 121, 122, 123, 0, 124, 125, 126, - 127, 128, 129, 130, 131, 999, 133, 1000, 1001, 0, - 136, 137, 138, 139, 140, 141, 1002, 599, 142, 143, - 1003, 1004, 146, 0, 147, 148, 149, 150, 600, 0, - 601, 0, 1005, 154, 155, 156, 157, 158, 159, 160, - 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, - 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 1006, 1007, 182, 1008, 183, 0, 184, 185, 186, - 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 764, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 1009, - 211, 212, 213, 214, 215, 602, 1010, 217, 0, 218, - 219, 1011, 221, 0, 222, 0, 223, 224, 0, 225, - 226, 227, 228, 229, 230, 0, 231, 0, 1012, 1013, - 234, 0, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 255, 256, 257, 258, 259, 260, - 261, 1014, 1015, 0, 1016, 0, 265, 266, 267, 268, - 269, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 277, 278, 279, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 1017, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 1018, 308, 1019, 310, 311, 312, 313, 1020, 314, - 315, 316, 317, 1021, 604, 319, 1022, 321, 322, 323, - 0, 324, 325, 0, 0, 1023, 327, 328, 0, 0, - 329, 330, 331, 332, 333, 606, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 607, 348, 349, 350, 351, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 1024, 365, 366, 367, 368, 0, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 384, 385, 386, 387, 1025, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 410, 411, 412, 1026, 414, 0, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, - 428, 609, 430, 0, 431, 432, 2592, 433, 434, 435, - 436, 437, 438, 439, 0, 1027, 1028, 0, 0, 442, - 443, 610, 445, 611, 1029, 447, 448, 612, 450, 451, - 452, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 1030, 0, 469, 470, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 1031, 0, 0, 0, 0, - 0, 0, 1032, 1033, 1034, 0, 0, 0, 0, 1035, - 0, 1036, 0, 0, 0, 0, 1037, 1038, 1039, 1040, - 109, 992, 629, 993, 994, 995, 996, 997, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, - 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, - 120, 0, 0, 0, 0, 0, 998, 0, 0, 121, - 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, - 131, 999, 133, 1000, 1001, 0, 136, 137, 138, 139, - 140, 141, 1002, 599, 142, 143, 1003, 1004, 146, 0, - 147, 148, 149, 150, 600, 0, 601, 0, 1005, 154, - 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, - 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, - 173, 174, 175, 176, 177, 178, 179, 1006, 1007, 182, - 1008, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 764, - 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 1009, 211, 212, 213, 214, - 215, 602, 1010, 217, 0, 218, 219, 1011, 221, 0, - 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, - 230, 0, 231, 0, 1012, 1013, 234, 0, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 255, 256, 257, 258, 259, 260, 261, 1014, 1015, 0, - 1016, 0, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 277, 278, 279, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 1017, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 1018, 308, 1019, - 310, 311, 312, 313, 1020, 314, 315, 316, 317, 1021, - 604, 319, 1022, 321, 322, 323, 0, 324, 325, 0, - 0, 1023, 327, 328, 0, 0, 329, 330, 331, 332, - 333, 606, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 607, 348, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 1024, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 384, - 385, 386, 387, 1025, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, - 1026, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 609, 430, 0, - 431, 432, 0, 433, 434, 435, 436, 437, 438, 439, - 0, 1027, 1028, 0, 0, 442, 443, 610, 445, 611, - 1029, 447, 448, 612, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 1030, 0, 469, 470, - 471, 472, 473, 474, 475, 0, 2700, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 1031, 0, 0, 0, 0, 0, 0, 1032, 1033, - 1034, 0, 0, 0, 0, 1035, 0, 1036, 0, 0, - 0, 0, 1037, 1038, 1039, 1040, 109, 992, 629, 993, - 994, 995, 996, 997, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 2910, 118, 119, 120, 0, 0, 0, - 0, 0, 998, 0, 0, 121, 122, 123, 0, 124, - 125, 126, 127, 128, 129, 130, 131, 999, 133, 1000, - 1001, 0, 136, 137, 138, 139, 140, 141, 1002, 599, - 142, 143, 1003, 1004, 146, 0, 147, 148, 149, 150, - 600, 0, 601, 0, 1005, 154, 155, 156, 157, 158, - 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 169, 170, 2911, 172, 173, 174, 175, 176, - 177, 178, 179, 1006, 1007, 182, 1008, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 764, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 1009, 211, 212, 213, 214, 215, 602, 1010, 217, - 0, 218, 219, 1011, 221, 0, 222, 0, 223, 224, - 0, 225, 226, 227, 228, 229, 230, 0, 231, 0, - 2912, 1013, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 255, 256, 257, 258, - 259, 260, 261, 1014, 1015, 0, 1016, 0, 265, 266, - 267, 268, 269, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 277, 278, 279, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 1017, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 1018, 308, 1019, 310, 311, 312, 313, - 1020, 314, 315, 316, 317, 1021, 604, 319, 1022, 321, - 322, 323, 0, 324, 325, 0, 0, 1023, 327, 328, - 0, 0, 329, 330, 331, 332, 333, 606, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 607, 348, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 1024, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 384, 385, 386, 2913, 1025, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 410, 411, 412, 1026, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 609, 430, 0, 431, 432, 0, 433, - 434, 435, 436, 437, 438, 439, 0, 1027, 1028, 0, - 0, 442, 443, 610, 445, 611, 1029, 447, 448, 612, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 1030, 0, 469, 470, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, + 302, 303, 304, 305, 306, 307, 1021, 309, 1022, 311, + 312, 313, 314, 1023, 315, 316, 317, 318, 1024, 607, + 320, 1025, 322, 323, 324, 0, 325, 326, 0, 0, + 1026, 328, 329, 0, 0, 330, 331, 332, 333, 334, + 335, 609, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 610, 350, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 1027, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 1028, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + 1029, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 612, 432, 0, + 433, 434, 0, 435, 436, 437, 438, 439, 440, 441, + 0, 442, 1030, 1031, 0, 0, 445, 446, 613, 448, + 614, 1032, 450, 451, 615, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 470, 1033, 0, 472, + 473, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 1034, 0, 0, 0, 0, 0, 0, 1035, + 1036, 1037, 0, 0, 0, 0, 1038, 0, 1039, 1990, + 0, 0, 0, 1040, 1041, 1042, 1043, 109, 995, 632, + 996, 997, 998, 999, 1000, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, + 124, 125, 126, 127, 128, 129, 130, 131, 1002, 133, + 1003, 1004, 0, 136, 137, 138, 139, 140, 141, 1005, + 602, 142, 143, 1006, 1007, 146, 0, 147, 148, 149, + 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, + 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, + 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, + 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, + 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, + 208, 209, 1012, 211, 212, 213, 214, 215, 605, 1013, + 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, + 224, 0, 225, 226, 227, 228, 229, 230, 0, 231, + 0, 1015, 1016, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 256, 257, + 258, 259, 260, 261, 262, 1017, 1018, 0, 1019, 0, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 278, 279, 280, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, + 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 1021, 309, 1022, 311, 312, + 313, 314, 1023, 315, 316, 317, 318, 1024, 607, 320, + 1025, 322, 323, 324, 0, 325, 326, 0, 0, 1026, + 328, 329, 0, 0, 330, 331, 332, 333, 334, 335, + 609, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 610, 350, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 1027, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, + 388, 389, 1028, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 1029, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 612, 432, 0, 433, + 434, 2595, 435, 436, 437, 438, 439, 440, 441, 0, + 442, 1030, 1031, 0, 0, 445, 446, 613, 448, 614, + 1032, 450, 451, 615, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 1033, 0, 472, 473, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, + 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 1034, 0, 0, 0, 0, 0, 0, 1035, 1036, + 1037, 0, 0, 0, 0, 1038, 0, 1039, 0, 0, + 0, 0, 1040, 1041, 1042, 1043, 109, 995, 632, 996, + 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, + 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, + 0, 0, 1001, 0, 0, 121, 122, 123, 0, 124, + 125, 126, 127, 128, 129, 130, 131, 1002, 133, 1003, + 1004, 0, 136, 137, 138, 139, 140, 141, 1005, 602, + 142, 143, 1006, 1007, 146, 0, 147, 148, 149, 150, + 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, + 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, + 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, + 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, + 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, + 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, + 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, + 209, 1012, 211, 212, 213, 214, 215, 605, 1013, 217, + 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, + 0, 225, 226, 227, 228, 229, 230, 0, 231, 0, + 1015, 1016, 234, 235, 0, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 256, 257, 258, + 259, 260, 261, 262, 1017, 1018, 0, 1019, 0, 266, + 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 278, 279, 280, 0, 281, 282, 283, + 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 1021, 309, 1022, 311, 312, 313, + 314, 1023, 315, 316, 317, 318, 1024, 607, 320, 1025, + 322, 323, 324, 0, 325, 326, 0, 0, 1026, 328, + 329, 0, 0, 330, 331, 332, 333, 334, 335, 609, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 610, 350, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 1027, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 386, 387, 388, + 389, 1028, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 412, 413, 414, 1029, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 429, 430, 612, 432, 0, 433, 434, + 0, 435, 436, 437, 438, 439, 440, 441, 0, 442, + 1030, 1031, 0, 0, 445, 446, 613, 448, 614, 1032, + 450, 451, 615, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 1033, 0, 472, 473, 474, + 475, 476, 477, 478, 0, 2703, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 1031, 0, 0, - 0, 0, 0, 0, 1032, 1033, 1034, 0, 0, 0, - 0, 1035, 0, 2914, 0, 0, 0, 0, 1037, 1038, - 1039, 1040, 109, 992, 629, 993, 994, 995, 996, 997, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 998, 0, - 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, - 129, 130, 131, 999, 133, 1000, 1001, 0, 136, 137, - 138, 139, 140, 141, 1002, 599, 142, 143, 1003, 1004, - 146, 0, 147, 148, 149, 150, 600, 0, 601, 0, - 1005, 154, 155, 156, 157, 158, 159, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 1006, - 1007, 182, 1008, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 764, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 1009, 211, 212, - 213, 214, 215, 602, 1010, 217, 0, 218, 219, 1011, - 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, - 228, 229, 230, 0, 231, 0, 1012, 1013, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 255, 256, 257, 258, 259, 260, 261, 1014, - 1015, 0, 1016, 0, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 277, 278, - 279, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 1017, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 1018, - 308, 1019, 310, 311, 312, 313, 1020, 314, 315, 316, - 317, 1021, 604, 319, 1022, 321, 322, 323, 0, 324, - 325, 0, 0, 1023, 327, 328, 0, 0, 329, 330, - 331, 332, 333, 606, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 607, 348, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 1024, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 384, 385, 386, 387, 1025, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, - 411, 412, 1026, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 609, - 430, 0, 431, 432, 3051, 433, 434, 435, 436, 437, - 438, 439, 0, 1027, 1028, 0, 0, 442, 443, 610, - 445, 611, 1029, 447, 448, 612, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 1030, 0, - 469, 470, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 1031, 0, 0, 0, 0, 0, 0, - 1032, 1033, 1034, 0, 0, 0, 0, 1035, 0, 1036, - 0, 0, 0, 0, 1037, 1038, 1039, 1040, 109, 992, - 629, 993, 994, 995, 996, 997, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, - 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 998, 0, 0, 121, 122, 123, - 0, 124, 125, 126, 127, 128, 129, 130, 131, 999, - 133, 1000, 1001, 0, 136, 137, 138, 139, 140, 141, - 1002, 599, 142, 143, 1003, 1004, 146, 0, 147, 148, - 149, 150, 600, 0, 601, 0, 1005, 154, 155, 156, - 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, - 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 1006, 1007, 182, 1008, 183, - 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 764, 198, 199, - 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 1009, 211, 212, 213, 214, 215, 602, - 1010, 217, 0, 218, 219, 1011, 221, 0, 222, 0, - 223, 224, 0, 225, 226, 227, 228, 229, 230, 0, - 231, 0, 1012, 1013, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 255, 256, - 257, 258, 259, 260, 261, 1014, 1015, 0, 1016, 0, - 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 277, 278, 279, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 1017, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 1018, 308, 1019, 310, 311, - 312, 313, 1020, 314, 315, 316, 317, 1021, 604, 319, - 1022, 321, 322, 323, 0, 324, 325, 0, 0, 1023, - 327, 328, 0, 0, 329, 330, 331, 332, 333, 606, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 607, 348, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 1024, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 384, 385, 386, - 387, 1025, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 410, 411, 412, 1026, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 609, 430, 0, 431, 432, - 3185, 433, 434, 435, 436, 437, 438, 439, 0, 1027, - 1028, 0, 0, 442, 443, 610, 445, 611, 1029, 447, - 448, 612, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 1030, 0, 469, 470, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 1031, - 0, 0, 0, 0, 0, 0, 1032, 1033, 1034, 0, - 0, 0, 0, 1035, 0, 1036, 0, 0, 0, 0, - 1037, 1038, 1039, 1040, 109, 992, 629, 993, 994, 995, - 996, 997, 0, 0, 0, 0, 0, 0, 0, 0, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 1034, 0, 0, 0, 0, 0, 0, 1035, 1036, 1037, + 0, 0, 0, 0, 1038, 0, 1039, 0, 0, 0, + 0, 1040, 1041, 1042, 1043, 109, 995, 632, 996, 997, + 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, + 116, 117, 2913, 118, 119, 120, 0, 0, 0, 0, + 0, 1001, 0, 0, 121, 122, 123, 0, 124, 125, + 126, 127, 128, 129, 130, 131, 1002, 133, 1003, 1004, + 0, 136, 137, 138, 139, 140, 141, 1005, 602, 142, + 143, 1006, 1007, 146, 0, 147, 148, 149, 150, 603, + 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, + 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, + 0, 169, 170, 2914, 172, 173, 174, 175, 176, 177, + 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, + 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, + 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, + 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, + 1012, 211, 212, 213, 214, 215, 605, 1013, 217, 0, + 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, + 225, 226, 227, 228, 229, 230, 0, 231, 0, 2915, + 1016, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 256, 257, 258, 259, + 260, 261, 262, 1017, 1018, 0, 1019, 0, 266, 267, + 268, 269, 270, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 278, 279, 280, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 1021, 309, 1022, 311, 312, 313, 314, + 1023, 315, 316, 317, 318, 1024, 607, 320, 1025, 322, + 323, 324, 0, 325, 326, 0, 0, 1026, 328, 329, + 0, 0, 330, 331, 332, 333, 334, 335, 609, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 610, 350, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 1027, 367, 368, 369, 370, 0, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 386, 387, 388, 2916, + 1028, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 412, 413, 414, 1029, 416, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 429, 430, 612, 432, 0, 433, 434, 0, + 435, 436, 437, 438, 439, 440, 441, 0, 442, 1030, + 1031, 0, 0, 445, 446, 613, 448, 614, 1032, 450, + 451, 615, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 1033, 0, 472, 473, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 1034, + 0, 0, 0, 0, 0, 0, 1035, 1036, 1037, 0, + 0, 0, 0, 1038, 0, 2917, 0, 0, 0, 0, + 1040, 1041, 1042, 1043, 109, 995, 632, 996, 997, 998, + 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 998, 0, 0, 121, 122, 123, 0, 124, 125, 126, - 127, 128, 129, 130, 131, 999, 133, 1000, 1001, 0, - 136, 137, 138, 139, 140, 141, 1002, 599, 142, 143, - 1003, 1004, 146, 0, 147, 148, 149, 150, 600, 0, - 601, 0, 1005, 154, 155, 156, 157, 158, 159, 160, + 1001, 0, 0, 121, 122, 123, 0, 124, 125, 126, + 127, 128, 129, 130, 131, 1002, 133, 1003, 1004, 0, + 136, 137, 138, 139, 140, 141, 1005, 602, 142, 143, + 1006, 1007, 146, 0, 147, 148, 149, 150, 603, 0, + 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 1006, 1007, 182, 1008, 183, 0, 184, 185, 186, + 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 764, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 1009, - 211, 212, 213, 214, 215, 602, 1010, 217, 0, 218, - 219, 1011, 221, 0, 222, 0, 223, 224, 0, 225, - 226, 227, 228, 229, 230, 0, 231, 0, 1012, 1013, - 234, 0, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 255, 256, 257, 258, 259, 260, - 261, 1014, 1015, 0, 1016, 0, 265, 266, 267, 268, - 269, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 277, 278, 279, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 1017, 289, 290, 291, 292, 293, 294, 295, + 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 1012, + 211, 212, 213, 214, 215, 605, 1013, 217, 0, 218, + 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, + 226, 227, 228, 229, 230, 0, 231, 0, 1015, 1016, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 256, 257, 258, 259, 260, + 261, 262, 1017, 1018, 0, 1019, 0, 266, 267, 268, + 269, 270, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 278, 279, 280, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 1018, 308, 1019, 310, 311, 312, 313, 1020, 314, - 315, 316, 317, 1021, 604, 319, 1022, 321, 322, 323, - 0, 324, 325, 0, 0, 1023, 327, 328, 0, 0, - 329, 330, 331, 332, 333, 606, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 607, 348, 349, 350, 351, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 1024, 365, 366, 367, 368, 0, 369, 370, 371, + 306, 307, 1021, 309, 1022, 311, 312, 313, 314, 1023, + 315, 316, 317, 318, 1024, 607, 320, 1025, 322, 323, + 324, 0, 325, 326, 0, 0, 1026, 328, 329, 0, + 0, 330, 331, 332, 333, 334, 335, 609, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 610, 350, 351, 352, 353, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 1027, 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 384, 385, 386, 387, 1025, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 410, 411, 412, 1026, 414, 0, 415, 416, 417, + 382, 383, 0, 384, 385, 386, 387, 388, 389, 1028, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, - 428, 609, 430, 0, 431, 432, 0, 433, 434, 435, - 436, 437, 438, 439, 0, 1027, 1028, 0, 0, 442, - 443, 610, 445, 611, 1029, 447, 448, 612, 450, 451, - 452, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 1030, 0, 469, 470, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, + 428, 429, 430, 612, 432, 0, 433, 434, 3054, 435, + 436, 437, 438, 439, 440, 441, 0, 442, 1030, 1031, + 0, 0, 445, 446, 613, 448, 614, 1032, 450, 451, + 615, 453, 454, 455, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 470, 1033, 0, 472, 473, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 1031, 0, 0, 0, 0, - 0, 0, 1032, 1033, 1034, 0, 0, 0, 0, 1035, - 0, 1036, 0, 0, 0, 0, 1037, 1038, 1039, 1040, - 109, 992, 629, 993, 994, 995, 996, 997, 0, 0, + 493, 494, 495, 496, 497, 498, 499, 500, 1034, 0, + 0, 0, 0, 0, 0, 1035, 1036, 1037, 0, 0, + 0, 0, 1038, 0, 1039, 0, 0, 0, 0, 1040, + 1041, 1042, 1043, 109, 995, 632, 996, 997, 998, 999, + 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, + 0, 118, 119, 120, 0, 0, 0, 0, 0, 1001, + 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, + 128, 129, 130, 131, 1002, 133, 1003, 1004, 0, 136, + 137, 138, 139, 140, 141, 1005, 602, 142, 143, 1006, + 1007, 146, 0, 147, 148, 149, 150, 603, 0, 604, + 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, + 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, + 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, + 203, 204, 205, 0, 206, 207, 208, 209, 1012, 211, + 212, 213, 214, 215, 605, 1013, 217, 0, 218, 219, + 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, + 227, 228, 229, 230, 0, 231, 0, 1015, 1016, 234, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 256, 257, 258, 259, 260, 261, + 262, 1017, 1018, 0, 1019, 0, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 278, 279, 280, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + 307, 1021, 309, 1022, 311, 312, 313, 314, 1023, 315, + 316, 317, 318, 1024, 607, 320, 1025, 322, 323, 324, + 0, 325, 326, 0, 0, 1026, 328, 329, 0, 0, + 330, 331, 332, 333, 334, 335, 609, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 610, 350, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 1027, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 1028, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 1029, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 429, 430, 612, 432, 0, 433, 434, 3188, 435, 436, + 437, 438, 439, 440, 441, 0, 442, 1030, 1031, 0, + 0, 445, 446, 613, 448, 614, 1032, 450, 451, 615, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 1033, 0, 472, 473, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, + 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 494, 495, 496, 497, 498, 499, 500, 1034, 0, 0, + 0, 0, 0, 0, 1035, 1036, 1037, 0, 0, 0, + 0, 1038, 0, 1039, 0, 0, 0, 0, 1040, 1041, + 1042, 1043, 109, 995, 632, 996, 997, 998, 999, 1000, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, + 118, 119, 120, 0, 0, 0, 0, 0, 1001, 0, + 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, + 129, 130, 131, 1002, 133, 1003, 1004, 0, 136, 137, + 138, 139, 140, 141, 1005, 602, 142, 143, 1006, 1007, + 146, 0, 147, 148, 149, 150, 603, 0, 604, 0, + 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, + 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 1009, + 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, + 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, + 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, + 204, 205, 0, 206, 207, 208, 209, 1012, 211, 212, + 213, 214, 215, 605, 1013, 217, 0, 218, 219, 1014, + 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, + 228, 229, 230, 0, 231, 0, 1015, 1016, 234, 235, + 0, 236, 237, 238, 239, 240, 241, 242, 243, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 256, 257, 258, 259, 260, 261, 262, + 1017, 1018, 0, 1019, 0, 266, 267, 268, 269, 270, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 278, + 279, 280, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 1021, 309, 1022, 311, 312, 313, 314, 1023, 315, 316, + 317, 318, 1024, 607, 320, 1025, 322, 323, 324, 0, + 325, 326, 0, 0, 1026, 328, 329, 0, 0, 330, + 331, 332, 333, 334, 335, 609, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 610, 350, 351, 352, 353, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 1027, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 386, 387, 388, 389, 1028, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 1029, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, + 430, 612, 432, 0, 433, 434, 0, 435, 436, 437, + 438, 439, 440, 441, 0, 442, 1030, 1031, 0, 0, + 445, 446, 613, 448, 614, 1032, 450, 451, 615, 453, + 454, 455, 456, 457, 0, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 470, 1033, 0, 472, 473, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, 499, 500, 1034, 0, 0, 0, + 0, 0, 0, 1035, 1036, 1037, 0, 0, 0, 0, + 1038, 0, 1039, 0, 0, 0, 0, 1040, 1041, 1042, + 1043, 109, 995, 632, 996, 997, 998, 999, 1000, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, + 119, 120, 0, 0, 0, 0, 0, 1001, 0, 0, + 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, + 130, 131, 1002, 133, 1003, 1004, 0, 136, 137, 138, + 139, 140, 141, 1005, 602, 142, 143, 1006, 1007, 146, + 0, 147, 148, 149, 150, 603, 0, 604, 0, 1008, + 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, + 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 1009, 1010, + 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 1012, 211, 212, 213, + 214, 215, 605, 1013, 217, 0, 218, 219, 1014, 221, + 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, + 229, 230, 0, 231, 0, 1015, 1016, 234, 235, 0, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 256, 257, 258, 259, 260, 261, 262, 1017, + 1018, 0, 1019, 0, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 278, 279, + 280, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 1021, + 309, 1022, 311, 312, 313, 314, 1023, 315, 316, 317, + 318, 1024, 607, 320, 1025, 322, 323, 324, 0, 325, + 326, 0, 0, 1026, 328, 329, 0, 0, 330, 331, + 332, 333, 334, 335, 609, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 610, 350, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 1027, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 1028, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 1029, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 612, 432, 0, 433, 434, 0, 435, 436, 437, 438, + 439, 440, 441, 0, 442, 1030, 1031, 0, 0, 445, + 446, 613, 448, 614, 1032, 450, 451, 615, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 1033, 0, 472, 473, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 1034, 0, 0, 0, 0, + 0, 0, 1663, 1664, 1037, 0, 0, 0, 0, 1038, + 0, 1039, 0, 0, 0, 0, 1040, 1041, 1042, 1043, + 109, 2122, 632, 996, 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, - 120, 0, 0, 0, 0, 0, 998, 0, 0, 121, + 120, 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, - 131, 999, 133, 1000, 1001, 0, 136, 137, 138, 139, - 140, 141, 1002, 599, 142, 143, 1003, 1004, 146, 0, - 147, 148, 149, 150, 600, 0, 601, 0, 1005, 154, + 131, 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, + 140, 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, + 147, 148, 149, 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, - 173, 174, 175, 176, 177, 178, 179, 1006, 1007, 182, - 1008, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 764, + 173, 174, 175, 176, 177, 178, 179, 1009, 1010, 182, + 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, + 190, 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 1009, 211, 212, 213, 214, - 215, 602, 1010, 217, 0, 218, 219, 1011, 221, 0, + 0, 206, 207, 208, 209, 1012, 211, 212, 213, 214, + 215, 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, - 230, 0, 231, 0, 1012, 1013, 234, 0, 235, 236, + 230, 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 255, 256, 257, 258, 259, 260, 261, 1014, 1015, 0, - 1016, 0, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 277, 278, 279, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 1017, 289, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 256, 257, 258, 259, 260, 261, 262, 1017, 1018, + 0, 1019, 0, 266, 267, 268, 269, 270, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 278, 279, 280, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 1018, 308, 1019, - 310, 311, 312, 313, 1020, 314, 315, 316, 317, 1021, - 604, 319, 1022, 321, 322, 323, 0, 324, 325, 0, - 0, 1023, 327, 328, 0, 0, 329, 330, 331, 332, - 333, 606, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 607, 348, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 1024, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 384, - 385, 386, 387, 1025, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, + 300, 301, 302, 303, 304, 305, 306, 307, 1021, 309, + 1022, 311, 312, 313, 314, 1023, 315, 316, 317, 318, + 1024, 607, 320, 1025, 322, 323, 324, 0, 325, 326, + 0, 0, 1026, 328, 329, 0, 0, 330, 331, 332, + 333, 334, 335, 609, 337, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 0, 0, 0, 0, 347, 348, + 610, 350, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 364, 365, 1027, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 1028, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, - 1026, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 609, 430, 0, - 431, 432, 0, 433, 434, 435, 436, 437, 438, 439, - 0, 1027, 1028, 0, 0, 442, 443, 610, 445, 611, - 1029, 447, 448, 612, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 1030, 0, 469, 470, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 413, 414, 1029, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 429, 430, 612, + 432, 0, 433, 434, 0, 435, 436, 437, 438, 439, + 440, 441, 0, 442, 1030, 1031, 0, 0, 445, 446, + 613, 448, 614, 1032, 450, 451, 615, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 0, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 470, 1033, + 0, 472, 473, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 1031, 0, 0, 0, 0, 0, 0, 1660, 1661, - 1034, 0, 0, 0, 0, 1035, 0, 1036, 0, 0, - 0, 0, 1037, 1038, 1039, 1040, 109, 2119, 629, 993, - 994, 995, 996, 997, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, - 0, 0, 998, 0, 0, 121, 122, 123, 0, 124, - 125, 126, 127, 128, 129, 130, 131, 999, 133, 1000, - 1001, 0, 136, 137, 138, 139, 140, 141, 1002, 599, - 142, 143, 1003, 1004, 146, 0, 147, 148, 149, 150, - 600, 0, 601, 0, 1005, 154, 155, 156, 157, 158, - 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, - 177, 178, 179, 1006, 1007, 182, 1008, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 764, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 1009, 211, 212, 213, 214, 215, 602, 1010, 217, - 0, 218, 219, 1011, 221, 0, 222, 0, 223, 224, - 0, 225, 226, 227, 228, 229, 230, 0, 231, 0, - 1012, 1013, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 255, 256, 257, 258, - 259, 260, 261, 1014, 1015, 0, 1016, 0, 265, 266, - 267, 268, 269, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 277, 278, 279, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 1017, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 1018, 308, 1019, 310, 311, 312, 313, - 1020, 314, 315, 316, 317, 1021, 604, 319, 1022, 321, - 322, 323, 0, 324, 325, 0, 0, 1023, 327, 328, - 0, 0, 329, 330, 331, 332, 333, 606, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 607, 348, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 1024, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 384, 385, 386, 387, 1025, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 410, 411, 412, 1026, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 609, 430, 0, 431, 432, 0, 433, - 434, 435, 436, 437, 438, 439, 0, 1027, 1028, 0, - 0, 442, 443, 610, 445, 611, 1029, 447, 448, 612, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 1030, 0, 469, 470, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 1031, 0, 0, - 0, 0, 0, 0, 1032, 1033, 1034, 0, 0, 0, - 0, 1035, 0, 1036, 0, 0, 0, 0, 1037, 1038, - 1039, 1040, 109, 992, 629, 993, 994, 995, 996, 997, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 998, 0, - 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, - 129, 130, 131, 999, 133, 1000, 1001, 0, 136, 137, - 138, 139, 140, 141, 1002, 599, 142, 143, 1003, 1004, - 146, 0, 147, 148, 149, 150, 600, 0, 601, 0, - 1005, 154, 155, 156, 157, 158, 159, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 1006, - 1007, 182, 1008, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 764, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 1009, 211, 212, - 213, 214, 215, 602, 1010, 217, 0, 218, 219, 1011, - 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, - 228, 229, 230, 0, 231, 0, 1012, 1013, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 255, 256, 257, 258, 259, 260, 261, 1014, - 1015, 0, 1016, 0, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 277, 278, - 279, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 1017, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 1018, - 308, 1019, 310, 311, 312, 313, 1020, 314, 315, 316, - 317, 1021, 604, 319, 1022, 321, 322, 323, 0, 324, - 325, 0, 0, 1023, 327, 328, 0, 0, 329, 330, - 331, 332, 333, 606, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 607, 348, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 1024, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 384, 385, 386, 387, 1025, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, - 411, 412, 1026, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 609, - 430, 0, 431, 432, 0, 433, 434, 435, 436, 437, - 438, 439, 0, 1027, 1028, 0, 0, 442, 443, 610, - 445, 611, 1029, 447, 448, 612, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 1030, 0, - 469, 470, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 1031, 0, 0, 0, 0, 0, 0, - 1032, 2201, 1034, 0, 0, 0, 0, 1035, 0, 1036, - 0, 0, 0, 0, 1037, 1038, 1039, 1040, 109, 992, - 629, 993, 994, 995, 996, 997, 0, 0, 0, 0, + 497, 498, 499, 500, 1034, 0, 0, 0, 0, 0, + 0, 1035, 1036, 1037, 0, 0, 0, 0, 1038, 0, + 1039, 0, 0, 0, 0, 1040, 1041, 1042, 1043, 109, + 995, 632, 996, 997, 998, 999, 1000, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, + 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, + 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, 140, + 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, + 148, 149, 150, 603, 0, 604, 0, 1008, 154, 155, + 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 1012, 211, 212, 213, 214, 215, + 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, + 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, + 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 256, 257, 258, 259, 260, 261, 262, 1017, 1018, 0, + 1019, 0, 266, 267, 268, 269, 270, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 278, 279, 280, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 1021, 309, 1022, + 311, 312, 313, 314, 1023, 315, 316, 317, 318, 1024, + 607, 320, 1025, 322, 323, 324, 0, 325, 326, 0, + 0, 1026, 328, 329, 0, 0, 330, 331, 332, 333, + 334, 335, 609, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 610, + 350, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 1027, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 1028, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 612, 432, + 0, 433, 434, 0, 435, 436, 437, 438, 439, 440, + 441, 0, 442, 1030, 1031, 0, 0, 445, 446, 613, + 448, 614, 1032, 450, 451, 615, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 1033, 0, + 472, 473, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 1034, 0, 0, 0, 0, 0, 0, + 1035, 2204, 1037, 0, 0, 0, 0, 1038, 0, 1039, + 0, 0, 0, 0, 1040, 1041, 1042, 1043, 109, 995, + 632, 996, 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 998, 0, 0, 121, 122, 123, - 0, 124, 125, 126, 127, 128, 129, 130, 131, 999, - 133, 1000, 1001, 0, 136, 137, 138, 139, 140, 141, - 1002, 599, 142, 143, 1003, 1004, 146, 0, 147, 148, - 149, 150, 600, 0, 601, 0, 1005, 154, 155, 156, + 0, 0, 0, 0, 1001, 0, 0, 121, 122, 123, + 0, 124, 125, 126, 127, 128, 129, 130, 131, 1002, + 133, 1003, 1004, 0, 136, 137, 138, 139, 140, 141, + 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, 148, + 149, 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 1006, 1007, 182, 1008, 183, + 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 764, 198, 199, + 192, 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 1009, 211, 212, 213, 214, 215, 602, - 1010, 217, 0, 218, 219, 1011, 221, 0, 222, 0, + 207, 208, 209, 1012, 211, 212, 213, 214, 215, 605, + 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, 0, - 231, 0, 1012, 1013, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 255, 256, - 257, 258, 259, 260, 261, 1014, 1015, 0, 1016, 0, - 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 277, 278, 279, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 1017, 289, 290, 291, + 231, 0, 1015, 1016, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 243, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 256, + 257, 258, 259, 260, 261, 262, 1017, 1018, 0, 1019, + 0, 266, 267, 268, 269, 270, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 278, 279, 280, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 1018, 308, 1019, 310, 311, - 312, 313, 1020, 314, 315, 316, 317, 1021, 604, 319, - 1022, 321, 322, 323, 0, 324, 325, 0, 0, 1023, - 327, 328, 0, 0, 329, 330, 331, 332, 333, 606, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 607, 348, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 1024, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 384, 385, 386, - 387, 1025, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 410, 411, 412, 1026, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 609, 430, 0, 431, 432, - 0, 433, 434, 435, 436, 437, 438, 439, 0, 1027, - 1028, 0, 0, 442, 443, 610, 445, 611, 1029, 447, - 448, 612, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 1030, 0, 469, 470, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 1031, - 0, 0, 0, 0, 0, 0, 1032, 1033, 1034, 0, - 0, 0, 0, 1035, 0, 2405, 0, 0, 0, 0, - 1037, 1038, 1039, 1040, 109, 992, 629, 993, 994, 995, - 996, 997, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, - 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 998, 0, 0, 121, 122, 123, 0, 124, 125, 126, - 127, 128, 129, 130, 131, 999, 133, 1000, 1001, 0, - 136, 137, 138, 139, 140, 141, 1002, 599, 142, 143, - 1003, 1004, 146, 0, 147, 148, 149, 150, 600, 0, - 601, 0, 1005, 154, 155, 156, 157, 158, 159, 160, - 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, - 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 1006, 1007, 182, 1008, 183, 0, 184, 185, 186, - 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 764, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 1009, - 211, 212, 213, 214, 215, 602, 1010, 217, 0, 218, - 219, 1011, 221, 0, 222, 0, 223, 224, 0, 225, - 226, 227, 228, 229, 230, 0, 231, 0, 1012, 1013, - 234, 0, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 255, 256, 257, 258, 259, 260, - 261, 1014, 1015, 0, 1016, 0, 265, 266, 267, 268, - 269, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 277, 278, 279, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 1017, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 1018, 308, 1019, 310, 311, 312, 313, 1020, 314, - 315, 316, 317, 1021, 604, 319, 1022, 321, 322, 323, - 0, 324, 325, 0, 0, 1023, 327, 328, 0, 0, - 329, 330, 331, 332, 333, 606, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 607, 348, 349, 350, 351, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 1024, 365, 366, 367, 368, 0, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 384, 385, 386, 387, 1025, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 410, 411, 412, 1026, 414, 0, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, - 428, 609, 430, 0, 431, 432, 0, 433, 434, 435, - 436, 437, 438, 439, 0, 1027, 1028, 0, 0, 442, - 443, 610, 445, 611, 1029, 447, 448, 612, 450, 451, - 452, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 1030, 0, 469, 470, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 1031, 0, 0, 0, 0, - 0, 0, 1032, 2958, 1034, 0, 0, 0, 0, 1035, - 0, 1036, 0, 0, 0, 0, 1037, 1038, 1039, 1040, - 109, 992, 629, 993, 994, 995, 996, 997, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, - 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, - 120, 0, 0, 0, 0, 0, 998, 0, 0, 121, - 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, - 131, 999, 133, 1000, 1001, 0, 136, 137, 138, 139, - 140, 141, 1002, 599, 142, 143, 1003, 1004, 146, 0, - 147, 148, 149, 150, 600, 0, 601, 0, 1005, 154, - 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, - 164, 165, 166, 167, 168, 0, 169, 170, 2911, 172, - 173, 174, 175, 176, 177, 178, 179, 1006, 1007, 182, - 1008, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 764, - 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 1009, 211, 212, 213, 214, - 215, 602, 1010, 217, 0, 218, 219, 1011, 221, 0, - 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, - 230, 0, 231, 0, 2912, 1013, 234, 0, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 255, 256, 257, 258, 259, 260, 261, 1014, 1015, 0, - 1016, 0, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 277, 278, 279, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 1017, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 1018, 308, 1019, - 310, 311, 312, 313, 1020, 314, 315, 316, 317, 1021, - 604, 319, 1022, 321, 322, 323, 0, 324, 325, 0, - 0, 1023, 327, 328, 0, 0, 329, 330, 331, 332, - 333, 606, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 607, 348, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 1024, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 384, - 385, 386, 2913, 1025, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, - 1026, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 609, 430, 0, - 431, 432, 0, 433, 434, 435, 436, 437, 438, 439, - 0, 1027, 1028, 0, 0, 442, 443, 610, 445, 611, - 1029, 447, 448, 612, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 1030, 0, 469, 470, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 1031, 0, 0, 0, 0, 0, 0, 1032, 1033, - 1034, 0, 0, 0, 0, 1035, 0, 2914, 0, 0, - 0, 0, 1037, 1038, 1039, 1040, 109, 992, 629, 993, - 994, 995, 996, 997, 0, 0, 0, 0, 0, 0, + 302, 303, 304, 305, 306, 307, 1021, 309, 1022, 311, + 312, 313, 314, 1023, 315, 316, 317, 318, 1024, 607, + 320, 1025, 322, 323, 324, 0, 325, 326, 0, 0, + 1026, 328, 329, 0, 0, 330, 331, 332, 333, 334, + 335, 609, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 610, 350, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 1027, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 1028, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + 1029, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 612, 432, 0, + 433, 434, 0, 435, 436, 437, 438, 439, 440, 441, + 0, 442, 1030, 1031, 0, 0, 445, 446, 613, 448, + 614, 1032, 450, 451, 615, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 470, 1033, 0, 472, + 473, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 1034, 0, 0, 0, 0, 0, 0, 1035, + 1036, 1037, 0, 0, 0, 0, 1038, 0, 2408, 0, + 0, 0, 0, 1040, 1041, 1042, 1043, 109, 995, 632, + 996, 997, 998, 999, 1000, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, + 124, 125, 126, 127, 128, 129, 130, 131, 1002, 133, + 1003, 1004, 0, 136, 137, 138, 139, 140, 141, 1005, + 602, 142, 143, 1006, 1007, 146, 0, 147, 148, 149, + 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, + 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, + 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, + 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, + 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, + 208, 209, 1012, 211, 212, 213, 214, 215, 605, 1013, + 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, + 224, 0, 225, 226, 227, 228, 229, 230, 0, 231, + 0, 1015, 1016, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 256, 257, + 258, 259, 260, 261, 262, 1017, 1018, 0, 1019, 0, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 278, 279, 280, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, + 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 1021, 309, 1022, 311, 312, + 313, 314, 1023, 315, 316, 317, 318, 1024, 607, 320, + 1025, 322, 323, 324, 0, 325, 326, 0, 0, 1026, + 328, 329, 0, 0, 330, 331, 332, 333, 334, 335, + 609, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 610, 350, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 1027, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, + 388, 389, 1028, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 1029, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 612, 432, 0, 433, + 434, 0, 435, 436, 437, 438, 439, 440, 441, 0, + 442, 1030, 1031, 0, 0, 445, 446, 613, 448, 614, + 1032, 450, 451, 615, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 1033, 0, 472, 473, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, + 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 1034, 0, 0, 0, 0, 0, 0, 1035, 2961, + 1037, 0, 0, 0, 0, 1038, 0, 1039, 0, 0, + 0, 0, 1040, 1041, 1042, 1043, 109, 995, 632, 996, + 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, - 0, 0, 998, 0, 0, 121, 122, 123, 0, 124, - 125, 126, 127, 128, 129, 130, 3358, 999, 133, 1000, - 1001, 0, 136, 137, 138, 139, 140, 141, 1002, 599, - 142, 143, 1003, 1004, 146, 0, 147, 148, 149, 150, - 600, 0, 601, 0, 1005, 154, 155, 156, 157, 158, + 0, 0, 1001, 0, 0, 121, 122, 123, 0, 124, + 125, 126, 127, 128, 129, 130, 131, 1002, 133, 1003, + 1004, 0, 136, 137, 138, 139, 140, 141, 1005, 602, + 142, 143, 1006, 1007, 146, 0, 147, 148, 149, 150, + 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 169, 170, 171, 3359, 173, 174, 175, 176, - 177, 178, 179, 1006, 1007, 182, 1008, 183, 0, 184, + 168, 0, 169, 170, 2914, 172, 173, 174, 175, 176, + 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 764, 198, 199, 0, 200, + 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 1009, 211, 212, 213, 214, 215, 602, 1010, 217, - 0, 218, 219, 1011, 221, 0, 222, 0, 223, 224, + 209, 1012, 211, 212, 213, 214, 215, 605, 1013, 217, + 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, 0, 231, 0, - 1012, 1013, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 255, 256, 257, 258, - 259, 260, 261, 1014, 1015, 0, 1016, 0, 265, 266, - 267, 268, 269, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 277, 278, 279, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 1017, 289, 290, 291, 292, 293, + 2915, 1016, 234, 235, 0, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 256, 257, 258, + 259, 260, 261, 262, 1017, 1018, 0, 1019, 0, 266, + 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 278, 279, 280, 0, 281, 282, 283, + 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 1018, 308, 1019, 310, 311, 312, 313, - 1020, 314, 315, 316, 317, 1021, 604, 319, 1022, 321, - 322, 323, 0, 324, 325, 0, 0, 1023, 327, 328, - 0, 0, 329, 330, 331, 332, 333, 606, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 607, 348, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 1024, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 384, 385, 386, 387, 1025, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 410, 411, 412, 1026, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 609, 430, 0, 431, 432, 0, 433, - 434, 435, 436, 437, 438, 439, 0, 1027, 1028, 0, - 0, 442, 443, 610, 445, 611, 1029, 447, 448, 612, - 450, 451, 3360, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 1030, 0, 469, 470, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, + 304, 305, 306, 307, 1021, 309, 1022, 311, 312, 313, + 314, 1023, 315, 316, 317, 318, 1024, 607, 320, 1025, + 322, 323, 324, 0, 325, 326, 0, 0, 1026, 328, + 329, 0, 0, 330, 331, 332, 333, 334, 335, 609, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 610, 350, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 1027, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 386, 387, 388, + 2916, 1028, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 412, 413, 414, 1029, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 429, 430, 612, 432, 0, 433, 434, + 0, 435, 436, 437, 438, 439, 440, 441, 0, 442, + 1030, 1031, 0, 0, 445, 446, 613, 448, 614, 1032, + 450, 451, 615, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 1033, 0, 472, 473, 474, + 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 1031, 0, 0, - 0, 0, 0, 0, 1032, 1033, 1034, 0, 0, 0, - 0, 1035, 0, 1036, 0, 0, 0, 0, 1037, 1038, - 1039, 1040, 109, 992, 629, 993, 994, 995, 996, 997, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 998, 0, - 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, - 129, 130, 131, 999, 133, 1000, 1001, 0, 136, 137, - 138, 139, 140, 141, 1002, 599, 142, 143, 1003, 1004, - 146, 0, 147, 148, 149, 150, 600, 0, 601, 0, - 1005, 154, 155, 156, 157, 158, 159, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, - 171, 3359, 173, 174, 175, 176, 177, 178, 179, 1006, - 1007, 182, 1008, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 764, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 1009, 211, 212, - 213, 214, 215, 602, 1010, 217, 0, 218, 219, 1011, - 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, - 228, 229, 230, 0, 231, 0, 1012, 1013, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 255, 256, 257, 258, 259, 260, 261, 1014, - 1015, 0, 1016, 0, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 277, 278, - 279, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 1017, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 1018, - 308, 1019, 310, 311, 312, 313, 1020, 314, 315, 316, - 317, 1021, 604, 319, 1022, 321, 322, 323, 0, 324, - 325, 0, 0, 1023, 327, 328, 0, 0, 329, 330, - 331, 332, 333, 606, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 607, 348, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 1024, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 384, 385, 386, 387, 1025, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, - 411, 412, 1026, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 609, - 430, 0, 431, 432, 0, 433, 434, 435, 436, 437, - 438, 439, 0, 1027, 1028, 0, 0, 442, 443, 610, - 445, 611, 1029, 447, 448, 612, 450, 451, 3360, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 1030, 0, - 469, 470, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 1031, 0, 0, 0, 0, 0, 0, - 1032, 1033, 1034, 0, 0, 0, 0, 1035, 0, 1036, - 0, 0, 0, 0, 1037, 1038, 1039, 1040, 109, 992, - 629, 993, 994, 995, 996, 997, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, - 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 998, 0, 0, 121, 122, 123, - 0, 124, 125, 126, 127, 128, 129, 130, 131, 999, - 133, 1000, 1001, 0, 136, 137, 138, 139, 140, 141, - 1002, 599, 142, 143, 1003, 1004, 146, 0, 147, 148, - 149, 150, 600, 0, 601, 0, 153, 154, 155, 156, - 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, - 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 1006, 1007, 182, 0, 183, - 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 764, 198, 199, - 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 1009, 211, 212, 213, 214, 215, 602, - 1010, 217, 0, 218, 219, 1011, 221, 0, 222, 0, - 223, 224, 0, 225, 226, 227, 228, 229, 230, 0, - 231, 0, 1012, 1013, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 255, 256, - 257, 258, 259, 260, 261, 1014, 1015, 0, 1016, 0, - 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 277, 278, 279, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 1017, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 1018, 308, 1019, 310, 311, - 312, 313, 0, 314, 315, 316, 317, 1021, 604, 319, - 1022, 321, 322, 323, 0, 324, 325, 0, 0, 1023, - 327, 328, 0, 0, 329, 330, 331, 332, 333, 606, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 607, 348, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 1024, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 384, 385, 386, - 387, 1025, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 410, 411, 412, 1026, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 609, 430, 0, 431, 432, - 0, 433, 434, 435, 436, 437, 438, 439, 0, 1027, - 1028, 0, 0, 442, 443, 610, 445, 611, 1029, 447, - 448, 612, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 1030, 0, 469, 470, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 0, - 0, 0, 0, 0, 0, 0, 1449, 1450, 0, 0, - 0, 0, 0, 1035, 0, 1036, 0, 0, 0, 0, - 1037, 1038, 1039, 1040, 109, 992, 629, 993, 994, 995, - 996, 997, 0, 0, 0, 0, 0, 0, 0, 0, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 1034, 0, 0, 0, 0, 0, 0, 1035, 1036, 1037, + 0, 0, 0, 0, 1038, 0, 2917, 0, 0, 0, + 0, 1040, 1041, 1042, 1043, 109, 995, 632, 996, 997, + 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, + 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, + 0, 1001, 0, 0, 121, 122, 123, 0, 124, 125, + 126, 127, 128, 129, 130, 3362, 1002, 133, 1003, 1004, + 0, 136, 137, 138, 139, 140, 141, 1005, 602, 142, + 143, 1006, 1007, 146, 0, 147, 148, 149, 150, 603, + 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, + 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, + 0, 169, 170, 171, 3363, 173, 174, 175, 176, 177, + 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, + 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, + 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, + 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, + 1012, 211, 212, 213, 214, 215, 605, 1013, 217, 0, + 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, + 225, 226, 227, 228, 229, 230, 0, 231, 0, 1015, + 1016, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 256, 257, 258, 259, + 260, 261, 262, 1017, 1018, 0, 1019, 0, 266, 267, + 268, 269, 270, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 278, 279, 280, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 1021, 309, 1022, 311, 312, 313, 314, + 1023, 315, 316, 317, 318, 1024, 607, 320, 1025, 322, + 323, 324, 0, 325, 326, 0, 0, 1026, 328, 329, + 0, 0, 330, 331, 332, 333, 334, 335, 609, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 610, 350, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 1027, 367, 368, 369, 370, 0, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 386, 387, 388, 389, + 1028, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 412, 413, 414, 1029, 416, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 429, 430, 612, 432, 0, 433, 434, 0, + 435, 436, 437, 438, 439, 440, 441, 0, 442, 1030, + 1031, 0, 0, 445, 446, 613, 448, 614, 1032, 450, + 451, 615, 453, 454, 3364, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 1033, 0, 472, 473, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 1034, + 0, 0, 0, 0, 0, 0, 1035, 1036, 1037, 0, + 0, 0, 0, 1038, 0, 1039, 0, 0, 0, 0, + 1040, 1041, 1042, 1043, 109, 995, 632, 996, 997, 998, + 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 998, 0, 0, 121, 122, 123, 0, 124, 125, 126, - 127, 128, 129, 130, -1968, 999, 133, 1000, 1001, 0, - 136, 137, 138, 139, 140, 141, 1002, 599, 142, 143, - 1003, 1004, 146, 0, 147, 148, 149, 150, 600, 0, - 601, 0, 1005, 154, 155, 156, 157, 158, 159, 160, + 1001, 0, 0, 121, 122, 123, 0, 124, 125, 126, + 127, 128, 129, 130, 131, 1002, 133, 1003, 1004, 0, + 136, 137, 138, 139, 140, 141, 1005, 602, 142, 143, + 1006, 1007, 146, 0, 147, 148, 149, 150, 603, 0, + 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, - 169, 170, 171, 3359, 173, 174, 175, 176, 177, 178, - 179, 1006, 1007, 182, 1008, 183, 0, 184, 185, 186, + 169, 170, 171, 3363, 173, 174, 175, 176, 177, 178, + 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 764, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 1009, - 211, 212, 213, 214, 215, 602, 1010, 217, 0, 218, - 219, 1011, 221, 0, 222, 0, 223, 224, 0, 225, - 226, 227, 228, -1968, 230, 0, 231, 0, 1012, 1013, - 234, 0, 235, 236, 237, 238, 239, 240, 241, -1968, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 255, 256, 257, 258, 259, 260, - 261, 1014, 1015, 0, 1016, 0, 265, 0, 0, 268, - 269, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 277, 278, -1968, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 1017, 289, 290, 291, 292, 293, 294, 295, + 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 1012, + 211, 212, 213, 214, 215, 605, 1013, 217, 0, 218, + 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, + 226, 227, 228, 229, 230, 0, 231, 0, 1015, 1016, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 256, 257, 258, 259, 260, + 261, 262, 1017, 1018, 0, 1019, 0, 266, 267, 268, + 269, 270, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 278, 279, 280, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 1018, 308, 1019, 310, 311, 312, 313, 0, 314, - 315, 0, 317, 1021, 604, 319, 1022, 321, 322, 323, - 0, 324, 325, 0, 0, 1023, 327, 328, 0, 0, - 329, 330, 331, 332, 333, 606, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 607, 348, 349, 350, 351, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 1024, 365, 366, 367, 368, 0, 369, 370, 371, + 306, 307, 1021, 309, 1022, 311, 312, 313, 314, 1023, + 315, 316, 317, 318, 1024, 607, 320, 1025, 322, 323, + 324, 0, 325, 326, 0, 0, 1026, 328, 329, 0, + 0, 330, 331, 332, 333, 334, 335, 609, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 610, 350, 351, 352, 353, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 1027, 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 384, 385, 386, 387, 1025, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1968, 411, 412, 1026, 414, 0, 415, 416, 417, + 382, 383, 0, 384, 385, 386, 387, 388, 389, 1028, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, - 428, 609, 430, 0, 431, 432, 0, 433, 434, 435, - 436, 437, 438, 439, 0, 1027, 1028, 0, 0, 442, - 443, 610, 445, 611, 1029, 447, 448, 612, 450, 451, - 3360, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 1030, 0, 469, 470, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, + 428, 429, 430, 612, 432, 0, 433, 434, 0, 435, + 436, 437, 438, 439, 440, 441, 0, 442, 1030, 1031, + 0, 0, 445, 446, 613, 448, 614, 1032, 450, 451, + 615, 453, 454, 3364, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 470, 1033, 0, 472, 473, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, -1968, 0, 0, 0, 0, - 0, 0, 1032, 1033, 1034, 0, 0, 0, 0, 1035, - 0, 1036, 0, 0, 0, 0, 1037, 1038, 1039, 1040, - 109, 992, 629, 993, 994, 0, 996, 997, 0, 0, + 493, 494, 495, 496, 497, 498, 499, 500, 1034, 0, + 0, 0, 0, 0, 0, 1035, 1036, 1037, 0, 0, + 0, 0, 1038, 0, 1039, 0, 0, 0, 0, 1040, + 1041, 1042, 1043, 109, 995, 632, 996, 997, 998, 999, + 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, + 0, 118, 119, 120, 0, 0, 0, 0, 0, 1001, + 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, + 128, 129, 130, 131, 1002, 133, 1003, 1004, 0, 136, + 137, 138, 139, 140, 141, 1005, 602, 142, 143, 1006, + 1007, 146, 0, 147, 148, 149, 150, 603, 0, 604, + 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 1009, 1010, 182, 0, 183, 0, 184, 185, 186, 187, + 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, + 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, + 203, 204, 205, 0, 206, 207, 208, 209, 1012, 211, + 212, 213, 214, 215, 605, 1013, 217, 0, 218, 219, + 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, + 227, 228, 229, 230, 0, 231, 0, 1015, 1016, 234, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 256, 257, 258, 259, 260, 261, + 262, 1017, 1018, 0, 1019, 0, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 278, 279, 280, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + 307, 1021, 309, 1022, 311, 312, 313, 314, 0, 315, + 316, 317, 318, 1024, 607, 320, 1025, 322, 323, 324, + 0, 325, 326, 0, 0, 1026, 328, 329, 0, 0, + 330, 331, 332, 333, 334, 335, 609, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 610, 350, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 1027, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 1028, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 1029, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 429, 430, 612, 432, 0, 433, 434, 0, 435, 436, + 437, 438, 439, 440, 441, 0, 442, 1030, 1031, 0, + 0, 445, 446, 613, 448, 614, 1032, 450, 451, 615, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 1033, 0, 472, 473, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, + 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 494, 495, 496, 497, 498, 499, 500, 0, 0, 0, + 0, 0, 0, 0, 1452, 1453, 0, 0, 0, 0, + 0, 1038, 0, 1039, 0, 0, 0, 0, 1040, 1041, + 1042, 1043, 109, 995, 632, 996, 997, 998, 999, 1000, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, + 118, 119, 120, 0, 0, 0, 0, 0, 1001, 0, + 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, + 129, 130, -1977, 1002, 133, 1003, 1004, 0, 136, 137, + 138, 139, 140, 141, 1005, 602, 142, 143, 1006, 1007, + 146, 0, 147, 148, 149, 150, 603, 0, 604, 0, + 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, + 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, + 171, 3363, 173, 174, 175, 176, 177, 178, 179, 1009, + 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, + 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, + 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, + 204, 205, 0, 206, 207, 208, 209, 1012, 211, 212, + 213, 214, 215, 605, 1013, 217, 0, 218, 219, 1014, + 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, + 228, -1977, 230, 0, 231, 0, 1015, 1016, 234, 235, + 0, 236, 237, 238, 239, 240, 241, 242, -1977, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 256, 257, 258, 259, 260, 261, 262, + 1017, 1018, 0, 1019, 0, 266, 0, 0, 269, 270, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 278, + 279, -1977, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 1021, 309, 1022, 311, 312, 313, 314, 0, 315, 316, + 0, 318, 1024, 607, 320, 1025, 322, 323, 324, 0, + 325, 326, 0, 0, 1026, 328, 329, 0, 0, 330, + 331, 332, 333, 334, 335, 609, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 610, 350, 351, 352, 353, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 1027, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 386, 387, 388, 389, 1028, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, -1977, 413, 414, 1029, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, + 430, 612, 432, 0, 433, 434, 0, 435, 436, 437, + 438, 439, 440, 441, 0, 442, 1030, 1031, 0, 0, + 445, 446, 613, 448, 614, 1032, 450, 451, 615, 453, + 454, 3364, 456, 457, 0, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 470, 1033, 0, 472, 473, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, 499, 500, -1977, 0, 0, 0, + 0, 0, 0, 1035, 1036, 1037, 0, 0, 0, 0, + 1038, 0, 1039, 0, 0, 0, 0, 1040, 1041, 1042, + 1043, 109, 995, 632, 996, 997, 0, 999, 1000, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, + 119, 120, 0, 0, 0, 0, 0, 1001, 0, 0, + 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, + 130, 131, 1002, 133, 1003, 1004, 0, 136, 137, 138, + 139, 140, 141, 1005, 602, 142, 143, 1006, 1007, 146, + 0, 147, 148, 149, 150, 603, 0, 604, 0, 153, + 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, + 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 1009, 1010, + 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 1012, 211, 212, 213, + 214, 215, 605, 1013, 217, 0, 218, 219, 1014, 221, + 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, + 229, 230, 0, 231, 0, 1015, 1016, 234, 235, 0, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 256, 257, 258, 259, 260, 261, 262, 1017, + 1018, 0, 1019, 0, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 278, 279, + 280, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 1021, + 309, 1022, 311, 312, 313, 314, 0, 315, 316, 317, + 318, 1024, 607, 320, 1025, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 332, 333, 334, 335, 609, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 610, 350, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 1027, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 2108, 2109, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 1029, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 612, 432, 0, 433, 434, 0, 435, 436, 437, 438, + 439, 440, 441, 0, 442, 1030, 1031, 0, 0, 445, + 446, 613, 448, 614, 1032, 450, 451, 615, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 1033, 0, 472, 473, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 0, 0, 0, 0, 0, + 0, 0, 2110, 2111, 0, 0, 0, 0, 0, 1038, + 0, 1039, 0, 0, 0, 0, 1040, 1041, 1042, 1043, + 109, 995, 632, 996, 997, 998, 999, 1000, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, - 120, 0, 0, 0, 0, 0, 998, 0, 0, 121, + 120, 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, - 131, 999, 133, 1000, 1001, 0, 136, 137, 138, 139, - 140, 141, 1002, 599, 142, 143, 1003, 1004, 146, 0, - 147, 148, 149, 150, 600, 0, 601, 0, 153, 154, + 131, 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, + 140, 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, + 147, 148, 149, 150, 603, 0, 604, 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, - 173, 174, 175, 176, 177, 178, 179, 1006, 1007, 182, + 173, 174, 175, 176, 177, 178, 179, 1009, 1010, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 764, + 190, 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 1009, 211, 212, 213, 214, - 215, 602, 1010, 217, 0, 218, 219, 1011, 221, 0, + 0, 206, 207, 208, 209, 1012, 211, 212, 213, 214, + 215, 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, - 230, 0, 231, 0, 1012, 1013, 234, 0, 235, 236, + 230, 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 255, 256, 257, 258, 259, 260, 261, 1014, 1015, 0, - 1016, 0, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 277, 278, 279, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 1017, 289, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 256, 257, 258, 259, 260, 261, 262, 1017, 1018, + 0, 1019, 0, 266, 0, 268, 269, 270, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 278, 279, 280, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 1018, 308, 1019, - 310, 311, 312, 313, 0, 314, 315, 316, 317, 1021, - 604, 319, 1022, 321, 322, 323, 0, 324, 325, 0, - 0, 326, 327, 328, 0, 0, 329, 330, 331, 332, - 333, 606, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 607, 348, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 1024, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 384, - 385, 386, 387, 2105, 2106, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, + 300, 301, 302, 303, 304, 305, 306, 307, 1021, 309, + 1022, 311, 312, 313, 314, 0, 315, 316, 317, 318, + 1024, 607, 320, 1025, 322, 323, 324, 0, 325, 326, + 0, 0, 1026, 328, 329, 0, 0, 330, 331, 332, + 333, 334, 335, 609, 337, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 0, 0, 0, 0, 347, 348, + 610, 350, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 364, 365, 1027, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 1028, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, - 1026, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 609, 430, 0, - 431, 432, 0, 433, 434, 435, 436, 437, 438, 439, - 0, 1027, 1028, 0, 0, 442, 443, 610, 445, 611, - 1029, 447, 448, 612, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 1030, 0, 469, 470, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 413, 414, 1029, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 429, 430, 612, + 432, 0, 433, 434, 0, 435, 436, 437, 438, 439, + 440, 441, 0, 442, 1030, 1031, 0, 0, 445, 446, + 613, 448, 614, 1032, 450, 451, 615, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 0, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 470, 1033, + 0, 472, 473, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 0, 0, 0, 0, 0, 0, 0, 2107, 2108, - 0, 0, 0, 0, 0, 1035, 0, 1036, 0, 0, - 0, 0, 1037, 1038, 1039, 1040, 109, 992, 629, 993, - 994, 995, 996, 997, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, - 0, 0, 998, 0, 0, 121, 122, 123, 0, 124, - 125, 126, 127, 128, 129, 130, 131, 999, 133, 1000, - 1001, 0, 136, 137, 138, 139, 140, 141, 1002, 599, - 142, 143, 1003, 1004, 146, 0, 147, 148, 149, 150, - 600, 0, 601, 0, 153, 154, 155, 156, 157, 158, - 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, - 177, 178, 179, 1006, 1007, 182, 0, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 764, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 1009, 211, 212, 213, 214, 215, 602, 1010, 217, - 0, 218, 219, 1011, 221, 0, 222, 0, 223, 224, - 0, 225, 226, 227, 228, 229, 230, 0, 231, 0, - 1012, 1013, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 255, 256, 257, 258, - 259, 260, 261, 1014, 1015, 0, 1016, 0, 265, 0, - 267, 268, 269, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 277, 278, 279, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 1017, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 1018, 308, 1019, 310, 311, 312, 313, - 0, 314, 315, 316, 317, 1021, 604, 319, 1022, 321, - 322, 323, 0, 324, 325, 0, 0, 1023, 327, 328, - 0, 0, 329, 330, 331, 332, 333, 606, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 607, 348, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 1024, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 384, 385, 386, 387, 1025, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 410, 411, 412, 1026, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 609, 430, 0, 431, 432, 0, 433, - 434, 435, 436, 437, 438, 439, 0, 1027, 1028, 0, - 0, 442, 443, 610, 445, 611, 1029, 447, 448, 612, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 1030, 0, 469, 470, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 0, 0, 0, - 0, 0, 0, 0, 1449, 1450, 0, 0, 0, 0, - 0, 1035, 0, 1036, 0, 0, 0, 0, 1037, 1038, - 1039, 1040, 109, 992, 629, 993, 994, 0, 996, 997, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 998, 0, - 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, - 129, 130, 131, 999, 133, 1000, 1001, 0, 136, 137, - 138, 139, 140, 141, 1002, 599, 142, 143, 1003, 1004, - 146, 0, 147, 148, 149, 150, 600, 0, 601, 0, - 153, 154, 155, 156, 157, 158, 159, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 1006, - 1007, 182, 0, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 764, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 1009, 211, 212, - 213, 214, 215, 602, 1010, 217, 0, 218, 219, 1011, - 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, - 228, 229, 230, 0, 231, 0, 1012, 1013, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 255, 256, 257, 258, 259, 260, 261, 1014, - 1015, 0, 1016, 0, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 277, 278, - 279, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 1017, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 1018, - 308, 1019, 310, 311, 312, 313, 0, 314, 315, 316, - 317, 1021, 604, 319, 1022, 321, 322, 323, 0, 324, - 325, 0, 0, 326, 327, 328, 0, 0, 329, 330, - 331, 332, 333, 606, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 607, 348, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 1024, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 384, 385, 386, 387, 1025, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, - 411, 412, 1026, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 609, - 430, 0, 431, 432, 0, 433, 434, 435, 436, 437, - 438, 439, 0, 1027, 1028, 0, 0, 442, 443, 610, - 445, 611, 1029, 447, 448, 612, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 1030, 0, - 469, 470, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 0, 0, 0, 0, 0, 0, 109, - 992, 629, 993, 994, 995, 996, 997, 1035, 0, 1036, - 0, 0, 0, 0, 1037, 1038, 1039, 1040, 110, 111, + 497, 498, 499, 500, 0, 0, 0, 0, 0, 0, + 0, 1452, 1453, 0, 0, 0, 0, 0, 1038, 0, + 1039, 0, 0, 0, 0, 1040, 1041, 1042, 1043, 109, + 995, 632, 996, 997, 0, 999, 1000, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, + 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, + 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, 140, + 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, + 148, 149, 150, 603, 0, 604, 0, 153, 154, 155, + 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 1009, 1010, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 1012, 211, 212, 213, 214, 215, + 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, + 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, + 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 256, 257, 258, 259, 260, 261, 262, 1017, 1018, 0, + 1019, 0, 266, 267, 268, 269, 270, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 278, 279, 280, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 1021, 309, 1022, + 311, 312, 313, 314, 0, 315, 316, 317, 318, 1024, + 607, 320, 1025, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 333, + 334, 335, 609, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 610, + 350, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 1027, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 1028, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 612, 432, + 0, 433, 434, 0, 435, 436, 437, 438, 439, 440, + 441, 0, 442, 1030, 1031, 0, 0, 445, 446, 613, + 448, 614, 1032, 450, 451, 615, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 1033, 0, + 472, 473, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 0, 0, 0, 0, 0, 0, 109, + 995, 632, 996, 997, 998, 999, 1000, 1038, 0, 1039, + 0, 0, 0, 0, 1040, 1041, 1042, 1043, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 0, 0, 998, 0, 0, 121, 122, + 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, 0, - 999, 133, 1000, 1001, 0, 136, 137, 138, 139, 140, - 141, 1002, 599, 142, 143, 1003, 1004, 146, 0, 147, - 148, 149, 150, 600, 0, 601, 0, 1005, 154, 155, + 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, 140, + 141, 1005, 602, 142, 143, 1006, 1007, 146, 0, 147, + 148, 149, 150, 603, 0, 604, 0, 1008, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 1006, 1007, 182, 1008, + 174, 175, 176, 177, 178, 179, 1009, 1010, 182, 1011, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 764, 198, + 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, 203, 0, 205, 0, - 206, 207, 208, 209, 1009, 211, 212, 213, 214, 215, - 602, 1010, 217, 0, 218, 219, 1011, 221, 0, 222, + 206, 207, 208, 209, 1012, 211, 212, 213, 214, 215, + 605, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, 0, 230, - 0, 231, 0, 1012, 1013, 234, 0, 235, 236, 237, - 238, 239, 240, 241, 0, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 255, - 256, 257, 258, 259, 260, 261, 1014, 1015, 0, 1016, - 0, 265, 0, 0, 268, 269, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 277, 278, 0, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 1017, 289, 290, + 0, 231, 0, 1015, 1016, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 0, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 256, 257, 258, 259, 260, 261, 262, 1017, 1018, 0, + 1019, 0, 266, 0, 0, 269, 270, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 278, 279, 0, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 1020, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 1018, 308, 1019, 310, - 311, 312, 313, 0, 314, 315, 0, 317, 1021, 604, - 319, 1022, 321, 322, 323, 0, 324, 325, 0, 0, - 1023, 327, 328, 0, 0, 329, 330, 331, 332, 333, - 606, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 607, 348, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 1024, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 384, 385, - 386, 387, 1025, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 0, 0, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 0, 411, 412, 1026, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 609, 430, 0, 431, - 432, 0, 433, 434, 435, 436, 437, 438, 439, 0, - 1027, 1028, 0, 0, 442, 443, 610, 445, 611, 1029, - 447, 448, 612, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 467, 1030, 0, 469, 470, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 301, 302, 303, 304, 305, 306, 307, 1021, 309, 1022, + 311, 312, 313, 314, 0, 315, 316, 0, 318, 1024, + 607, 320, 1025, 322, 323, 324, 0, 325, 326, 0, + 0, 1026, 328, 329, 0, 0, 330, 331, 332, 333, + 334, 335, 609, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 610, + 350, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 1027, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 1028, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 0, 413, + 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 612, 432, + 0, 433, 434, 0, 435, 436, 437, 438, 439, 440, + 441, 0, 442, 1030, 1031, 0, 0, 445, 446, 613, + 448, 614, 1032, 450, 451, 615, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 1033, 0, + 472, 473, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 0, 0, 0, 0, 0, 0, 0, 1032, 1033, 1034, - 0, 834, 1408, 629, 1035, 0, 1036, 996, 0, 0, - 0, 1037, 1038, 1039, 1040, 0, 0, 0, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, - 119, 120, 0, 0, 0, 835, 0, 0, 0, 0, - 836, 122, 123, 0, 124, 125, 126, 837, 128, 129, - 130, 838, 839, 840, 841, 842, 0, 136, 137, 138, - 139, 140, 141, 0, 0, 142, 143, 843, 844, 146, - 0, 147, 148, 149, 150, 845, 0, 846, 0, 847, - 154, 155, 156, 157, 158, 848, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 849, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 850, 851, - 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, - 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 852, 211, 212, 213, - 214, 215, 853, 1409, 217, 0, 218, 219, 854, 221, - 0, 222, 0, 223, 855, 0, 856, 226, 227, 857, - 858, 230, 0, 231, 0, 859, 860, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 861, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 862, 863, 257, 258, 259, 260, 261, 864, 865, - 0, 866, 0, 265, 867, 868, 268, 869, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 870, 278, 871, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 872, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 873, 874, - 875, 310, 311, 312, 876, 0, 314, 315, 877, 317, - 0, 878, 319, 879, 321, 322, 323, 0, 324, 325, - 1410, 0, 326, 327, 328, 0, 0, 329, 880, 881, - 332, 882, 883, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 884, - 885, 349, 350, 886, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 887, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 888, 385, 386, 387, 889, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 890, 400, 401, - 402, 403, 404, 405, 891, 407, 408, 409, 892, 411, - 412, 893, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 894, 428, 895, 430, - 0, 431, 432, 0, 433, 896, 435, 436, 437, 438, - 439, 0, 897, 898, 0, 0, 442, 443, 899, 445, - 900, 1411, 447, 448, 901, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 1251, 903, 0, 469, - 904, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 905, 906, 907, - 908, 909, 910, 911, 912, 913, 914, 915, 494, 495, - 496, 497, 0, 0, 0, 0, 0, 0, 0, 1412, - 1413, 2293, 0, 109, 992, 629, 993, 994, 2294, 996, - 0, 0, 0, 0, 0, 0, 1040, 0, 0, 0, - 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, - 0, 118, 119, 120, 0, 0, 0, 0, 0, 998, - 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, - 128, 129, 130, 131, 999, 133, 1000, 1001, 0, 136, - 137, 138, 139, 140, 141, 1002, 0, 142, 143, 1003, - 1004, 146, 0, 147, 148, 149, 150, 151, 0, 152, - 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, - 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, + 498, 499, 500, 0, 0, 0, 0, 0, 0, 0, + 1035, 1036, 1037, 0, 837, 1411, 632, 1038, 0, 1039, + 999, 0, 0, 0, 1040, 1041, 1042, 1043, 0, 0, + 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, + 117, 0, 118, 119, 120, 0, 0, 0, 838, 0, + 0, 0, 0, 839, 122, 123, 0, 124, 125, 126, + 840, 128, 129, 130, 841, 842, 843, 844, 845, 0, + 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, + 846, 847, 146, 0, 147, 148, 149, 150, 848, 0, + 849, 0, 850, 154, 155, 156, 157, 158, 851, 160, + 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, + 852, 170, 171, 172, 173, 174, 175, 176, 177, 178, + 179, 853, 854, 182, 0, 183, 0, 184, 185, 186, + 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, + 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 855, + 211, 212, 213, 214, 215, 856, 1412, 217, 0, 218, + 219, 857, 221, 0, 222, 0, 223, 858, 0, 859, + 226, 227, 860, 861, 230, 0, 231, 0, 862, 863, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 864, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 865, 866, 258, 259, 260, + 261, 262, 867, 868, 0, 869, 0, 266, 870, 871, + 269, 872, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 873, 279, 874, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 875, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 876, 877, 878, 311, 312, 313, 879, 0, + 315, 316, 880, 318, 0, 881, 320, 882, 322, 323, + 324, 0, 325, 326, 1413, 0, 327, 328, 329, 0, + 0, 330, 331, 883, 884, 334, 885, 886, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 887, 888, 351, 352, 889, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 890, 367, 368, 369, 370, 0, 371, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 891, 387, 388, 389, 892, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 893, 402, 403, 404, 405, 406, 407, 894, + 409, 410, 411, 895, 413, 414, 896, 416, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, + 428, 897, 430, 898, 432, 0, 433, 434, 0, 435, + 899, 437, 438, 439, 440, 441, 0, 442, 900, 901, + 0, 0, 445, 446, 902, 448, 903, 1414, 450, 451, + 904, 453, 454, 455, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 1254, 906, 0, 472, 907, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, + 483, 484, 485, 908, 909, 910, 911, 912, 913, 914, + 915, 916, 917, 918, 497, 498, 499, 500, 0, 0, + 0, 0, 0, 0, 0, 1415, 1416, 2296, 0, 109, + 995, 632, 996, 997, 2297, 999, 0, 0, 0, 0, + 0, 0, 1043, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 1001, 0, 0, 121, 122, + 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, + 1002, 133, 1003, 1004, 0, 136, 137, 138, 139, 140, + 141, 1005, 0, 142, 143, 1006, 1007, 146, 0, 147, + 148, 149, 150, 151, 0, 152, 0, 153, 154, 155, + 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 1009, 1010, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 1012, 211, 212, 213, 214, 215, + 216, 1013, 217, 0, 218, 219, 1014, 221, 0, 222, + 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, + 0, 231, 2934, 1015, 1016, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 256, 257, 258, 259, 260, 261, 262, 1017, 1018, 0, + 1019, 0, 266, 267, 268, 269, 270, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 278, 279, 280, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 1021, 309, 1022, + 311, 312, 313, 314, 0, 315, 316, 317, 318, 1024, + 319, 320, 1025, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 333, + 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 349, + 350, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 1027, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 1029, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 431, 432, + 0, 433, 434, 0, 435, 436, 437, 438, 439, 440, + 441, 0, 442, 1030, 1031, 0, 0, 445, 446, 447, + 448, 449, 1032, 450, 451, 452, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 1033, 0, + 472, 473, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 0, 0, 0, 0, 109, 995, 632, + 996, 997, 0, 999, 0, 0, 0, 1038, 0, 2652, + 0, 0, 0, 0, 1040, 1041, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 1001, 0, 0, 121, 122, 123, 0, + 124, 125, 126, 127, 128, 129, 130, 131, 1002, 133, + 1003, 1004, 0, 136, 137, 138, 139, 140, 141, 1005, + 0, 142, 143, 1006, 1007, 146, 0, 147, 148, 149, + 150, 151, 0, 152, 0, 153, 154, 155, 156, 157, + 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 1009, 1010, 182, 0, 183, 0, + 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, + 193, 194, 0, 0, 195, 196, 767, 198, 199, 0, + 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, + 208, 209, 1012, 211, 212, 213, 214, 215, 216, 1013, + 217, 0, 218, 219, 1014, 221, 0, 222, 0, 223, + 224, 0, 225, 226, 227, 228, 229, 230, 0, 231, + 0, 1015, 1016, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 256, 257, + 258, 259, 260, 261, 262, 1017, 1018, 0, 1019, 0, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 278, 279, 280, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, + 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 1021, 309, 1022, 311, 312, + 313, 314, 0, 315, 316, 317, 318, 1024, 319, 320, + 1025, 322, 323, 324, 0, 325, 326, 0, 0, 327, + 328, 329, 0, 0, 330, 331, 332, 333, 334, 335, + 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 1027, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 1029, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 431, 432, 0, 433, + 434, 0, 435, 436, 437, 438, 439, 440, 441, 0, + 442, 1030, 1031, 0, 0, 445, 446, 447, 448, 449, + 1032, 450, 451, 452, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 1033, 0, 472, 473, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, + 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 0, 0, 0, 0, 508, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1038, 0, 2652, 0, 0, + 0, 0, 1040, 1041, 110, 111, 112, 113, 114, 115, + 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, + 0, 0, 953, 0, 0, 122, 123, 0, 124, 125, + 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, + 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, + 143, 144, 145, 146, 0, 147, 148, 149, 150, 151, + 0, 0, 0, 153, 154, 155, 156, 157, 158, 0, + 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, + 0, 0, 170, 171, 172, 173, 174, 175, 176, 177, + 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, + 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, + 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, + 202, 0, 203, 204, 205, -503, 206, 207, 208, 209, + 210, 211, 212, 213, 214, 215, 216, 0, 217, -503, + 218, 219, 220, 221, -503, 222, 0, 223, 0, 0, + 0, 226, 227, 509, 0, 230, 0, 231, 0, 232, + 233, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 0, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 0, 257, 258, 259, + 260, 261, 262, 263, 264, -503, 265, 0, 266, 0, + 0, 269, 0, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 0, 279, 0, -503, 281, 282, 283, 284, + 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 0, 310, 311, 312, 313, 314, + 0, 315, 316, 0, 318, 0, 319, 320, 321, 322, + 323, 324, -503, 325, 326, 0, 0, 327, 328, 329, + 0, -503, 330, 331, 332, 0, 334, 0, 336, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 349, 0, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 366, 367, 368, 369, 370, 0, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 0, 413, 414, 415, 416, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 511, 430, 431, 432, 0, 433, 434, 0, + 435, 0, 437, 438, 439, 440, 441, 0, 442, 443, + 444, 0, 0, 445, 446, 447, 448, 449, 0, 450, + 451, 452, 453, 454, 455, 456, 457, -503, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 471, 0, 472, 0, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 508, + 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1108, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, + 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, + 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, + 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, + 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, + 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, + 0, 223, 0, 0, 0, 226, 227, 509, 0, 230, + 0, 231, 0, 232, 233, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 0, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 0, 257, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 266, 0, 0, 269, 0, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 0, 279, 0, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 0, 310, + 311, 312, 313, 314, 0, 315, 316, 0, 318, 0, + 319, 320, 321, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 0, + 334, 0, 336, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 349, + 0, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 366, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 0, 413, + 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 511, 430, 431, 432, + 0, 433, 434, 0, 435, 0, 437, 438, 439, 440, + 441, 0, 442, 443, 444, 0, 0, 445, 446, 447, + 448, 449, 0, 450, 451, 452, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 471, 0, + 472, 0, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 837, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2383, + 3107, 0, 110, 111, 112, 113, 114, 115, 116, 117, + 0, 118, 119, 120, 3, 4, 0, 838, 0, 0, + 0, 0, 839, 122, 123, 0, 124, 125, 126, 840, + 128, 129, 130, 841, 842, 843, 844, 845, 0, 136, + 137, 138, 139, 140, 141, 0, 0, 142, 143, 846, + 847, 146, 0, 147, 148, 149, 150, 848, 0, 849, + 0, 850, 154, 155, 156, 157, 158, 851, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 852, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, - 1006, 1007, 182, 0, 183, 0, 184, 185, 186, 187, + 853, 854, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, - 195, 196, 764, 198, 199, 0, 200, 201, 202, 0, - 203, 204, 205, 0, 206, 207, 208, 209, 1009, 211, - 212, 213, 214, 215, 216, 1010, 217, 0, 218, 219, - 1011, 221, 0, 222, 0, 223, 224, 0, 225, 226, - 227, 228, 229, 230, 0, 231, 2931, 1012, 1013, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 255, 256, 257, 258, 259, 260, 261, - 1014, 1015, 0, 1016, 0, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 277, - 278, 279, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, + 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, + 203, 204, 205, 0, 206, 207, 208, 209, 855, 211, + 212, 213, 214, 215, 856, 0, 217, 0, 218, 219, + 857, 221, 0, 222, 0, 223, 858, 21, 859, 226, + 227, 860, 861, 230, 0, 231, 0, 862, 863, 234, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 864, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 865, 866, 258, 259, 260, 261, + 262, 867, 868, 0, 869, 0, 266, 870, 871, 269, + 872, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 873, 279, 874, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 875, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 1018, 308, 1019, 310, 311, 312, 313, 0, 314, 315, - 316, 317, 1021, 318, 319, 1022, 321, 322, 323, 0, - 324, 325, 0, 0, 326, 327, 328, 0, 0, 329, - 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 1024, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 410, 411, 412, 1026, 414, 0, 415, 416, 417, 418, + 307, 876, 877, 878, 311, 312, 313, 879, 0, 315, + 316, 880, 318, 0, 881, 320, 882, 322, 323, 324, + 0, 325, 326, 0, 0, 327, 328, 329, 0, 0, + 330, 331, 883, 884, 334, 885, 886, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 25, 26, 27, + 0, 347, 348, 887, 888, 351, 352, 889, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 890, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 891, 387, 388, 389, 892, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 32, 893, 402, 403, 404, 405, 406, 407, 894, 409, + 410, 411, 895, 413, 414, 896, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 429, 430, 0, 431, 432, 0, 433, 434, 435, 436, - 437, 438, 439, 0, 1027, 1028, 0, 0, 442, 443, - 444, 445, 446, 1029, 447, 448, 449, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 1030, - 0, 469, 470, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 0, 0, 0, 0, 109, 992, - 629, 993, 994, 0, 996, 0, 0, 0, 1035, 0, - 2649, 0, 0, 0, 0, 1037, 1038, 110, 111, 112, + 897, 430, 898, 432, 0, 433, 434, 37, 435, 899, + 437, 438, 439, 440, 441, 0, 442, 900, 901, 0, + 0, 445, 446, 902, 448, 903, 0, 450, 451, 904, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 39, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 905, 906, 0, 472, 907, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 43, 480, 481, 482, 483, + 484, 485, 908, 909, 910, 911, 912, 913, 914, 915, + 916, 917, 918, 497, 498, 499, 500, 0, 109, 44, + 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 45, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 998, 0, 0, 121, 122, 123, - 0, 124, 125, 126, 127, 128, 129, 130, 131, 999, - 133, 1000, 1001, 0, 136, 137, 138, 139, 140, 141, - 1002, 0, 142, 143, 1003, 1004, 146, 0, 147, 148, - 149, 150, 151, 0, 152, 0, 153, 154, 155, 156, + 0, 0, 0, 0, 0, 0, 0, 121, 122, 123, + 0, 124, 125, 126, 127, 128, 129, 130, 131, 132, + 133, 134, 135, 0, 136, 137, 138, 139, 140, 141, + 0, 602, 142, 143, 144, 145, 146, 0, 147, 148, + 149, 150, 603, 0, 604, 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 1006, 1007, 182, 0, 183, + 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 764, 198, 199, + 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 1009, 211, 212, 213, 214, 215, 216, - 1010, 217, 0, 218, 219, 1011, 221, 0, 222, 0, - 223, 224, 0, 225, 226, 227, 228, 229, 230, 0, - 231, 0, 1012, 1013, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 255, 256, - 257, 258, 259, 260, 261, 1014, 1015, 0, 1016, 0, - 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 277, 278, 279, 0, 280, 281, + 207, 208, 209, 210, 211, 212, 213, 214, 215, 605, + 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, + 223, 224, 21, 225, 226, 227, 228, 229, 230, 0, + 231, 0, 232, 233, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 243, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 256, + 257, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 266, 267, 268, 269, 270, 271, 272, 273, 274, + 275, 276, 606, 0, 277, 278, 279, 280, 0, 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 1018, 308, 1019, 310, 311, - 312, 313, 0, 314, 315, 316, 317, 1021, 318, 319, - 1022, 321, 322, 323, 0, 324, 325, 0, 0, 326, - 327, 328, 0, 0, 329, 330, 331, 332, 333, 334, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 1024, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 410, 411, 412, 1026, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 429, 430, 0, 431, 432, - 0, 433, 434, 435, 436, 437, 438, 439, 0, 1027, - 1028, 0, 0, 442, 443, 444, 445, 446, 1029, 447, - 448, 449, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 1030, 0, 469, 470, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 0, - 0, 0, 0, 505, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1035, 0, 2649, 0, 0, 0, 0, - 1037, 1038, 110, 111, 112, 113, 114, 115, 116, 117, + 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, + 312, 313, 314, 0, 315, 316, 317, 318, 0, 607, + 320, 321, 322, 323, 324, 0, 325, 326, 0, 608, + 327, 328, 329, 0, 0, 330, 331, 332, 333, 334, + 335, 609, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 25, 26, 27, 0, 347, 348, 610, 350, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 366, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 611, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 32, 0, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 612, 432, 0, + 433, 434, 37, 435, 436, 437, 438, 439, 440, 441, + 0, 442, 443, 444, 0, 0, 445, 446, 613, 448, + 614, 0, 450, 451, 615, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 39, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 616, 471, 0, 472, + 473, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 43, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 0, 109, 44, 530, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 617, 0, + 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, - 950, 0, 0, 122, 123, 0, 124, 125, 126, 0, - 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, - 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, - 145, 146, 0, 147, 148, 149, 150, 151, 0, 0, - 0, 153, 154, 155, 156, 157, 158, 0, 160, 161, - 162, 0, 163, 164, 165, 166, 167, 168, 0, 0, + 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, + 128, 129, 130, 131, 132, 133, 134, 135, 0, 136, + 137, 138, 139, 140, 141, 0, 602, 142, 143, 144, + 145, 146, 0, 147, 148, 149, 150, 603, 0, 604, + 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, - 203, 204, 205, -503, 206, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, 0, 217, -503, 218, 219, - 220, 221, -503, 222, 0, 223, 0, 0, 0, 226, - 227, 506, 0, 230, 0, 231, 0, 232, 233, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 0, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 0, 256, 257, 258, 259, 260, 261, - 262, 263, -503, 264, 0, 265, 0, 0, 268, 0, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 0, - 278, 0, -503, 280, 281, 282, 283, 284, 285, 286, - 287, 507, 289, 290, 291, 292, 293, 294, 295, 296, + 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 605, 0, 217, 0, 218, 219, + 220, 221, 0, 222, 0, 223, 224, 0, 225, 226, + 227, 228, 229, 230, 0, 231, 0, 232, 233, 234, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 606, 0, 277, + 278, 279, 280, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 0, 309, 310, 311, 312, 313, 0, 314, 315, - 0, 317, 0, 318, 319, 320, 321, 322, 323, -503, - 324, 325, 0, 0, 326, 327, 328, 0, -503, 329, - 330, 0, 332, 0, 334, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 347, 0, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 364, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 0, 385, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 0, 411, 412, 413, 414, 0, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 508, 428, - 429, 430, 0, 431, 432, 0, 433, 0, 435, 436, - 437, 438, 439, 0, 440, 441, 0, 0, 442, 443, - 444, 445, 446, 0, 447, 448, 449, 450, 451, 452, - 453, 454, -503, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 468, - 0, 469, 0, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, + 307, 308, 309, 310, 311, 312, 313, 314, 0, 315, + 316, 317, 318, 0, 607, 320, 321, 322, 323, 324, + 0, 325, 326, 0, 608, 327, 328, 329, 0, 0, + 330, 331, 332, 333, 334, 335, 609, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 610, 350, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 611, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 415, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 429, 430, 612, 432, 0, 433, 434, 0, 435, 436, + 437, 438, 439, 440, 441, 0, 442, 443, 444, 0, + 0, 445, 446, 613, 448, 614, 0, 450, 451, 615, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 616, 471, 0, 472, 473, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 505, 0, 527, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1105, 0, 110, 111, 112, 113, 114, 115, 116, - 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, - 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, - 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, - 144, 145, 146, 0, 147, 148, 149, 150, 151, 0, - 0, 0, 153, 154, 155, 156, 157, 158, 0, 160, - 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, - 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, - 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, - 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, - 226, 227, 506, 0, 230, 0, 231, 0, 232, 233, - 234, 0, 235, 236, 237, 238, 239, 240, 241, 0, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 0, 256, 257, 258, 259, 260, - 261, 262, 263, 0, 264, 0, 265, 0, 0, 268, - 0, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 0, 278, 0, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 507, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 0, 309, 310, 311, 312, 313, 0, 314, - 315, 0, 317, 0, 318, 319, 320, 321, 322, 323, - 0, 324, 325, 0, 0, 326, 327, 328, 0, 0, - 329, 330, 0, 332, 0, 334, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 347, 0, 349, 350, 351, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 364, 365, 366, 367, 368, 0, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 0, 385, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 0, 411, 412, 413, 414, 0, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 508, - 428, 429, 430, 0, 431, 432, 0, 433, 0, 435, - 436, 437, 438, 439, 0, 440, 441, 0, 0, 442, - 443, 444, 445, 446, 0, 447, 448, 449, 450, 451, - 452, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 468, 0, 469, 0, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 834, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 2380, 3104, 0, 110, 111, 112, 113, 114, 115, - 116, 117, 0, 118, 119, 120, 3, 4, 0, 835, - 0, 0, 0, 0, 836, 122, 123, 0, 124, 125, - 126, 837, 128, 129, 130, 838, 839, 840, 841, 842, - 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, - 143, 843, 844, 146, 0, 147, 148, 149, 150, 845, - 0, 846, 0, 847, 154, 155, 156, 157, 158, 848, - 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, - 0, 849, 170, 171, 172, 173, 174, 175, 176, 177, - 178, 179, 850, 851, 182, 0, 183, 0, 184, 185, - 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, - 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, - 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, - 852, 211, 212, 213, 214, 215, 853, 0, 217, 0, - 218, 219, 854, 221, 0, 222, 0, 223, 855, 21, - 856, 226, 227, 857, 858, 230, 0, 231, 0, 859, - 860, 234, 0, 235, 236, 237, 238, 239, 240, 241, - 861, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 862, 863, 257, 258, 259, - 260, 261, 864, 865, 0, 866, 0, 265, 867, 868, - 268, 869, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 870, 278, 871, 0, 280, 281, 282, 283, 284, - 285, 286, 287, 872, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 873, 874, 875, 310, 311, 312, 876, 0, - 314, 315, 877, 317, 0, 878, 319, 879, 321, 322, - 323, 0, 324, 325, 0, 0, 326, 327, 328, 0, - 0, 329, 880, 881, 332, 882, 883, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 25, 26, 27, - 0, 345, 346, 884, 885, 349, 350, 886, 352, 353, - 354, 0, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 887, 365, 366, 367, 368, 0, 369, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 888, 385, 386, 387, 889, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 32, 890, 400, 401, 402, 403, 404, 405, 891, 407, - 408, 409, 892, 411, 412, 893, 414, 0, 415, 416, - 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 894, 428, 895, 430, 0, 431, 432, 37, 433, 896, - 435, 436, 437, 438, 439, 0, 897, 898, 0, 0, - 442, 443, 899, 445, 900, 0, 447, 448, 901, 450, - 451, 452, 453, 454, 0, 0, 455, 456, 457, 39, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 902, 903, 0, 469, 904, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 43, 477, 478, 479, 480, 481, - 482, 905, 906, 907, 908, 909, 910, 911, 912, 913, - 914, 915, 494, 495, 496, 497, 0, 109, 44, 527, + 494, 495, 496, 497, 498, 499, 500, 109, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 45, 0, 0, 0, 110, 111, 112, 113, + 0, 0, 0, 617, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, 132, 133, 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, - 599, 142, 143, 144, 145, 146, 0, 147, 148, 149, - 150, 600, 0, 601, 0, 153, 154, 155, 156, 157, + 602, 142, 143, 144, 145, 146, 0, 147, 148, 149, + 150, 603, 0, 604, 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 602, 0, + 208, 209, 210, 211, 212, 213, 214, 215, 605, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 21, 225, 226, 227, 228, 229, 230, 0, 231, - 0, 232, 233, 234, 0, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 255, 256, 257, - 258, 259, 260, 261, 262, 263, 0, 264, 0, 265, + 224, 0, 225, 226, 227, 228, 229, 230, 0, 231, + 0, 232, 233, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 256, 257, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, - 603, 0, 276, 277, 278, 279, 0, 280, 281, 282, + 276, 0, 0, 277, 278, 279, 280, 0, 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 0, 314, 315, 316, 317, 0, 604, 319, 320, - 321, 322, 323, 0, 324, 325, 0, 605, 326, 327, - 328, 0, 0, 329, 330, 331, 332, 333, 606, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 25, - 26, 27, 0, 345, 346, 607, 348, 349, 350, 351, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 364, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 384, 385, 386, 387, - 388, 608, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 32, 0, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 410, 411, 412, 413, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 609, 430, 0, 431, 432, 37, - 433, 434, 435, 436, 437, 438, 439, 0, 440, 441, - 0, 0, 442, 443, 610, 445, 611, 0, 447, 448, - 612, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 39, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 613, 468, 0, 469, 470, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 43, 477, 478, 479, + 313, 314, 0, 315, 316, 317, 318, 0, 607, 320, + 321, 322, 323, 324, 0, 325, 326, 0, 0, 327, + 328, 329, 0, 0, 330, 331, 332, 333, 334, 335, + 609, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 610, 350, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 366, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, + 388, 389, 390, 611, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 612, 432, 0, 433, + 434, 0, 435, 436, 437, 438, 439, 440, 441, 0, + 442, 443, 444, 0, 0, 445, 446, 613, 448, 614, + 0, 450, 451, 615, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 471, 0, 472, 473, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 0, 109, - 44, 527, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 0, 110, 111, - 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 0, 0, 0, 0, 0, 121, 122, - 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, - 132, 133, 134, 135, 0, 136, 137, 138, 139, 140, - 141, 0, 599, 142, 143, 144, 145, 146, 0, 147, - 148, 149, 150, 600, 0, 601, 0, 153, 154, 155, - 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, - 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, - 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, - 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, - 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 602, 0, 217, 0, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, - 0, 231, 0, 232, 233, 234, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 255, - 256, 257, 258, 259, 260, 261, 262, 263, 0, 264, - 0, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 274, 275, 603, 0, 276, 277, 278, 279, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 0, 314, 315, 316, 317, 0, 604, - 319, 320, 321, 322, 323, 0, 324, 325, 0, 605, - 326, 327, 328, 0, 0, 329, 330, 331, 332, 333, - 606, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 607, 348, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 364, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 384, 385, - 386, 387, 388, 608, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 0, 0, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 609, 430, 0, 431, - 432, 0, 433, 434, 435, 436, 437, 438, 439, 0, - 440, 441, 0, 0, 442, 443, 610, 445, 611, 0, - 447, 448, 612, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 613, 468, 0, 469, 470, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 109, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 0, 110, - 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, - 120, 0, 0, 0, 0, 0, 0, 0, 0, 121, - 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, - 131, 132, 133, 134, 135, 0, 136, 137, 138, 139, - 140, 141, 0, 599, 142, 143, 144, 145, 146, 0, - 147, 148, 149, 150, 600, 0, 601, 0, 153, 154, - 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, - 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, - 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, - 215, 602, 0, 217, 0, 218, 219, 220, 221, 0, - 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, - 230, 0, 231, 0, 232, 233, 234, 0, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 255, 256, 257, 258, 259, 260, 261, 262, 263, 0, - 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 277, 278, 279, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 288, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 0, 314, 315, 316, 317, 0, - 604, 319, 320, 321, 322, 323, 0, 324, 325, 0, - 0, 326, 327, 328, 0, 0, 329, 330, 331, 332, - 333, 606, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 607, 348, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 364, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 384, - 385, 386, 387, 388, 608, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, - 413, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 609, 430, 0, - 431, 432, 0, 433, 434, 435, 436, 437, 438, 439, - 0, 440, 441, 0, 0, 442, 443, 610, 445, 611, - 0, 447, 448, 612, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 468, 0, 469, 470, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 109, 0, 0, 0, 0, 0, 0, 0, 0, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 109, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 45, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, 132, 133, 134, 135, 0, 136, 137, 138, - 139, 140, 141, 0, 599, 142, 143, 144, 145, 146, - 0, 147, 148, 149, 150, 600, 0, 601, 0, 153, + 139, 140, 141, 0, 602, 142, 143, 144, 145, 146, + 0, 147, 148, 149, 150, 603, 0, 604, 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, @@ -7288,139 +7329,43 @@ static const yytype_int16 yytable[] = 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 602, 0, 217, 0, 218, 219, 220, 221, + 214, 215, 605, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, - 229, 230, 0, 231, 0, 232, 233, 234, 0, 235, + 229, 230, 0, 231, 0, 232, 233, 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, - 0, 264, 0, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 277, 278, 279, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 288, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 256, 257, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 0, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 278, 279, + 280, 0, 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 0, 314, 315, 316, 317, - 0, 604, 319, 320, 321, 322, 323, 0, 324, 325, - 0, 0, 326, 327, 328, 0, 0, 329, 330, 331, - 332, 333, 606, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 607, - 348, 349, 350, 351, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 364, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, + 309, 310, 311, 312, 313, 314, 0, 315, 316, 317, + 318, 0, 607, 320, 321, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 332, 333, 334, 335, 609, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 610, 350, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 0, 400, 401, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, - 412, 413, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 427, 428, 609, 430, - 0, 431, 432, 0, 433, 434, 435, 436, 437, 438, - 439, 0, 440, 441, 0, 0, 442, 443, 610, 445, - 611, 0, 447, 448, 612, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 467, 468, 0, 469, - 470, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 412, 413, 414, 415, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 612, 432, 0, 433, 434, 0, 435, 436, 437, 438, + 439, 440, 441, 0, 442, 443, 444, 0, 0, 445, + 446, 613, 448, 614, 0, 450, 451, 615, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 471, 0, 472, 473, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 505, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3188, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 699, - 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, - 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, - 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, - 146, 0, 147, 148, 149, 150, 151, 0, 0, 0, - 153, 154, 155, 156, 157, 158, 0, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 0, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, - 221, 0, 222, 0, 223, 0, 21, 0, 226, 227, - 506, 0, 230, 0, 231, 0, 232, 233, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 0, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 0, 256, 257, 258, 259, 260, 261, 262, - 263, 0, 264, 0, 265, 0, 0, 268, 0, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 0, 278, - 0, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 507, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 0, 309, 310, 311, 312, 313, 0, 314, 315, 0, - 317, 0, 318, 319, 320, 321, 322, 323, 0, 324, - 325, 0, 0, 326, 327, 328, 0, 0, 329, 330, - 0, 332, 0, 334, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 25, 26, 27, 0, 345, 346, - 347, 0, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 364, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 0, 385, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 32, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 0, - 411, 412, 413, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 508, 428, 429, - 430, 0, 431, 432, 37, 433, 0, 435, 436, 437, - 438, 439, 0, 700, 441, 0, 0, 701, 443, 444, - 445, 446, 0, 447, 448, 449, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 39, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 613, 468, 0, - 469, 0, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 43, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 505, 0, 44, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 45, - 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, - 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, - 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, - 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, - 145, 146, 0, 147, 148, 149, 150, 151, 0, 0, - 0, 153, 154, 155, 156, 157, 158, 0, 160, 161, - 162, 0, 163, 164, 165, 166, 167, 168, 0, 0, - 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, - 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, - 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, - 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, - 220, 221, 0, 222, 0, 223, 0, 21, 0, 226, - 227, 506, 0, 230, 0, 231, 0, 232, 233, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 0, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 0, 256, 257, 258, 259, 260, 261, - 262, 263, 0, 264, 0, 265, 0, 0, 268, 0, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 0, - 278, 0, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 507, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 0, 309, 310, 311, 312, 313, 0, 314, 315, - 0, 317, 0, 318, 319, 320, 321, 322, 323, 0, - 324, 325, 0, 0, 326, 327, 328, 0, 0, 329, - 330, 0, 332, 0, 334, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 25, 26, 27, 0, 345, - 346, 347, 0, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 364, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 0, 385, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 32, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 0, 411, 412, 413, 414, 0, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 508, 428, - 429, 430, 0, 431, 432, 37, 433, 0, 435, 436, - 437, 438, 439, 0, 440, 441, 0, 0, 442, 443, - 444, 445, 446, 0, 447, 448, 449, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 39, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 613, 468, - 0, 469, 0, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 43, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 0, 505, 44, 527, 0, 0, + 496, 497, 498, 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 45, 0, 0, 0, 110, 111, 112, 113, 114, 115, - 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, + 0, 3191, 0, 0, 110, 111, 112, 113, 114, 115, + 116, 117, 702, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, @@ -7433,185 +7378,41 @@ static const yytype_int16 yytable[] = 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, - 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, - 0, 226, 227, 506, 0, 230, 0, 231, 0, 232, - 233, 234, 0, 235, 236, 237, 238, 239, 240, 241, - 0, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 0, 256, 257, 258, 259, - 260, 261, 262, 263, 0, 264, 0, 265, 0, 0, - 268, 0, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 0, 278, 0, 0, 280, 281, 282, 283, 284, - 285, 286, 287, 507, 289, 290, 291, 292, 293, 294, + 218, 219, 220, 221, 0, 222, 0, 223, 0, 21, + 0, 226, 227, 509, 0, 230, 0, 231, 0, 232, + 233, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 0, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 0, 257, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 266, 0, + 0, 269, 0, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 0, 279, 0, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 0, 309, 310, 311, 312, 313, 0, - 314, 315, 0, 317, 0, 318, 319, 320, 321, 322, - 323, 0, 324, 325, 0, 0, 326, 327, 328, 0, - 0, 329, 330, 0, 332, 0, 334, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 0, 0, 0, - 0, 345, 346, 347, 0, 349, 350, 351, 352, 353, - 354, 0, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 364, 365, 366, 367, 368, 0, 369, 370, + 305, 306, 307, 308, 0, 310, 311, 312, 313, 314, + 0, 315, 316, 0, 318, 0, 319, 320, 321, 322, + 323, 324, 0, 325, 326, 0, 0, 327, 328, 329, + 0, 0, 330, 331, 332, 0, 334, 0, 336, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 25, + 26, 27, 0, 347, 348, 349, 0, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 0, 385, 386, 387, 388, 389, + 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 0, 0, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, 0, 411, 412, 413, 414, 0, 415, 416, + 400, 401, 32, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 508, 428, 429, 430, 0, 431, 432, 0, 433, 0, - 435, 436, 437, 438, 439, 0, 440, 441, 0, 0, - 442, 443, 444, 445, 446, 0, 447, 448, 449, 450, - 451, 452, 453, 454, 0, 0, 455, 456, 457, 0, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 467, 468, 0, 469, 0, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 0, 477, 478, 479, 480, 481, + 427, 428, 511, 430, 431, 432, 0, 433, 434, 37, + 435, 0, 437, 438, 439, 440, 441, 0, 442, 703, + 444, 0, 0, 704, 446, 447, 448, 449, 0, 450, + 451, 452, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 39, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 616, 471, 0, 472, 0, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 43, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 505, 0, 527, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 935, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, - 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, - 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, - 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, - 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, - 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, - 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, - 0, 0, 226, 227, 506, 0, 230, 0, 231, 0, - 232, 233, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 0, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 0, 256, 257, 258, - 259, 260, 261, 262, 263, 0, 264, 0, 265, 0, - 0, 268, 0, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 0, 278, 0, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 507, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 0, 309, 310, 311, 312, 313, - 0, 314, 315, 0, 317, 0, 318, 319, 320, 321, - 322, 323, 0, 324, 325, 0, 0, 326, 327, 328, - 0, 0, 329, 330, 0, 332, 0, 334, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 347, 0, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 364, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 0, 385, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 0, 411, 412, 413, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 508, 428, 429, 430, 0, 431, 432, 0, 433, - 0, 435, 436, 437, 438, 439, 0, 440, 441, 0, - 0, 442, 443, 444, 445, 446, 0, 447, 448, 449, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 468, 0, 469, 0, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 505, 0, 527, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1503, 0, 0, 110, 111, 112, 113, - 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, - 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, - 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, - 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, - 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, - 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, - 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, - 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, - 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, - 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, - 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, - 0, 0, 0, 226, 227, 506, 0, 230, 0, 231, - 0, 232, 233, 234, 0, 235, 236, 237, 238, 239, - 240, 241, 0, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 0, 256, 257, - 258, 259, 260, 261, 262, 263, 0, 264, 0, 265, - 0, 0, 268, 0, 270, 271, 272, 273, 274, 275, - 0, 0, 276, 0, 278, 0, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 507, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 0, 309, 310, 311, 312, - 313, 0, 314, 315, 0, 317, 0, 318, 319, 320, - 321, 322, 323, 0, 324, 325, 0, 0, 326, 327, - 328, 0, 0, 329, 330, 0, 332, 0, 334, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 347, 0, 349, 350, 351, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 364, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 0, 385, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 0, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 0, 411, 412, 413, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 508, 428, 429, 430, 0, 431, 432, 0, - 433, 0, 435, 436, 437, 438, 439, 0, 440, 441, - 0, 0, 442, 443, 444, 445, 446, 0, 447, 448, - 449, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 467, 468, 0, 469, 0, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 505, 0, - 527, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2131, 0, 0, 110, 111, 112, - 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, - 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, - 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, - 0, 0, 142, 143, 144, 145, 146, 0, 147, 148, - 149, 150, 151, 0, 0, 0, 153, 154, 155, 156, - 157, 158, 0, 160, 161, 162, 0, 163, 164, 165, - 166, 167, 168, 0, 0, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, - 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, - 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, - 223, 0, 0, 0, 226, 227, 506, 0, 230, 0, - 231, 0, 232, 233, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 0, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 0, 256, - 257, 258, 259, 260, 261, 262, 263, 0, 264, 0, - 265, 0, 0, 268, 0, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 0, 278, 0, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 507, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 0, 309, 310, 311, - 312, 313, 0, 314, 315, 0, 317, 0, 318, 319, - 320, 321, 322, 323, 0, 324, 325, 0, 0, 326, - 327, 328, 0, 0, 329, 330, 0, 332, 0, 334, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 347, 0, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 364, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 0, 385, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 0, 411, 412, 413, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 508, 428, 429, 430, 0, 431, 432, - 0, 433, 0, 435, 436, 437, 438, 439, 0, 440, - 441, 0, 0, 442, 443, 444, 445, 446, 0, 447, - 448, 449, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 468, 0, 469, 0, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 505, - 0, 527, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2380, 0, 0, 110, 111, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 508, + 0, 44, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 45, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, @@ -7626,40 +7427,234 @@ static const yytype_int16 yytable[] = 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, - 0, 223, 0, 0, 0, 226, 227, 506, 0, 230, - 0, 231, 0, 232, 233, 234, 0, 235, 236, 237, - 238, 239, 240, 241, 0, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 0, - 256, 257, 258, 259, 260, 261, 262, 263, 0, 264, - 0, 265, 0, 0, 268, 0, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 0, 278, 0, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 507, 289, 290, + 0, 223, 0, 21, 0, 226, 227, 509, 0, 230, + 0, 231, 0, 232, 233, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 0, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 0, 257, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 266, 0, 0, 269, 0, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 0, 279, 0, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 0, 309, 310, - 311, 312, 313, 0, 314, 315, 0, 317, 0, 318, - 319, 320, 321, 322, 323, 0, 324, 325, 0, 0, - 326, 327, 328, 0, 0, 329, 330, 0, 332, 0, - 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 347, 0, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 364, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 0, 385, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 0, 0, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 0, 411, 412, 413, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 508, 428, 429, 430, 0, 431, - 432, 0, 433, 0, 435, 436, 437, 438, 439, 0, - 440, 441, 0, 0, 442, 443, 444, 445, 446, 0, - 447, 448, 449, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 467, 468, 0, 469, 0, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 301, 302, 303, 304, 305, 306, 307, 308, 0, 310, + 311, 312, 313, 314, 0, 315, 316, 0, 318, 0, + 319, 320, 321, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 0, + 334, 0, 336, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 25, 26, 27, 0, 347, 348, 349, + 0, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 366, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 32, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 0, 413, + 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 511, 430, 431, 432, + 0, 433, 434, 37, 435, 0, 437, 438, 439, 440, + 441, 0, 442, 443, 444, 0, 0, 445, 446, 447, + 448, 449, 0, 450, 451, 452, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 39, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 616, 471, 0, + 472, 0, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 43, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 505, 0, 527, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2488, 0, 0, 110, + 498, 499, 500, 0, 508, 44, 530, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 45, + 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, + 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, + 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, + 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, + 144, 145, 146, 0, 147, 148, 149, 150, 151, 0, + 0, 0, 153, 154, 155, 156, 157, 158, 0, 160, + 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, + 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, + 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, + 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, + 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, + 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, + 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, + 226, 227, 509, 0, 230, 0, 231, 0, 232, 233, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 0, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 0, 257, 258, 259, 260, + 261, 262, 263, 264, 0, 265, 0, 266, 0, 0, + 269, 0, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 0, 279, 0, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 0, 310, 311, 312, 313, 314, 0, + 315, 316, 0, 318, 0, 319, 320, 321, 322, 323, + 324, 0, 325, 326, 0, 0, 327, 328, 329, 0, + 0, 330, 331, 332, 0, 334, 0, 336, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 349, 0, 351, 352, 353, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 366, 367, 368, 369, 370, 0, 371, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 0, 413, 414, 415, 416, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, + 428, 511, 430, 431, 432, 0, 433, 434, 0, 435, + 0, 437, 438, 439, 440, 441, 0, 442, 443, 444, + 0, 0, 445, 446, 447, 448, 449, 0, 450, 451, + 452, 453, 454, 455, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 470, 471, 0, 472, 0, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 498, 499, 500, 508, 0, + 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 938, 0, 0, 110, 111, 112, + 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, + 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, + 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, + 0, 0, 142, 143, 144, 145, 146, 0, 147, 148, + 149, 150, 151, 0, 0, 0, 153, 154, 155, 156, + 157, 158, 0, 160, 161, 162, 0, 163, 164, 165, + 166, 167, 168, 0, 0, 170, 171, 172, 173, 174, + 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, + 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, + 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, + 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, + 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, + 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, + 223, 0, 0, 0, 226, 227, 509, 0, 230, 0, + 231, 0, 232, 233, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 0, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 0, + 257, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 266, 0, 0, 269, 0, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 0, 279, 0, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, + 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 0, 310, 311, + 312, 313, 314, 0, 315, 316, 0, 318, 0, 319, + 320, 321, 322, 323, 324, 0, 325, 326, 0, 0, + 327, 328, 329, 0, 0, 330, 331, 332, 0, 334, + 0, 336, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 349, 0, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 366, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 0, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 0, 413, 414, + 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 511, 430, 431, 432, 0, + 433, 434, 0, 435, 0, 437, 438, 439, 440, 441, + 0, 442, 443, 444, 0, 0, 445, 446, 447, 448, + 449, 0, 450, 451, 452, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 470, 471, 0, 472, + 0, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1506, 0, + 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, + 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, + 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, + 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, + 146, 0, 147, 148, 149, 150, 151, 0, 0, 0, + 153, 154, 155, 156, 157, 158, 0, 160, 161, 162, + 0, 163, 164, 165, 166, 167, 168, 0, 0, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, + 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, + 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, + 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, + 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, + 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, + 221, 0, 222, 0, 223, 0, 0, 0, 226, 227, + 509, 0, 230, 0, 231, 0, 232, 233, 234, 235, + 0, 236, 237, 238, 239, 240, 241, 242, 0, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 0, 257, 258, 259, 260, 261, 262, + 263, 264, 0, 265, 0, 266, 0, 0, 269, 0, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 0, + 279, 0, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 0, 310, 311, 312, 313, 314, 0, 315, 316, + 0, 318, 0, 319, 320, 321, 322, 323, 324, 0, + 325, 326, 0, 0, 327, 328, 329, 0, 0, 330, + 331, 332, 0, 334, 0, 336, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 349, 0, 351, 352, 353, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 366, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 0, 413, 414, 415, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 511, + 430, 431, 432, 0, 433, 434, 0, 435, 0, 437, + 438, 439, 440, 441, 0, 442, 443, 444, 0, 0, + 445, 446, 447, 448, 449, 0, 450, 451, 452, 453, + 454, 455, 456, 457, 0, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 470, 471, 0, 472, 0, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, 499, 500, 508, 0, 530, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 2134, 0, 0, 110, 111, 112, 113, 114, + 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, + 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, + 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, + 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, + 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, + 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, + 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, + 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, + 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, + 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, + 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, + 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, + 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, + 0, 0, 226, 227, 509, 0, 230, 0, 231, 0, + 232, 233, 234, 235, 0, 236, 237, 238, 239, 240, + 241, 242, 0, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 0, 257, 258, + 259, 260, 261, 262, 263, 264, 0, 265, 0, 266, + 0, 0, 269, 0, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 0, 279, 0, 0, 281, 282, 283, + 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 0, 310, 311, 312, 313, + 314, 0, 315, 316, 0, 318, 0, 319, 320, 321, + 322, 323, 324, 0, 325, 326, 0, 0, 327, 328, + 329, 0, 0, 330, 331, 332, 0, 334, 0, 336, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 349, 0, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 366, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 0, 413, 414, 415, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 511, 430, 431, 432, 0, 433, 434, + 0, 435, 0, 437, 438, 439, 440, 441, 0, 442, + 443, 444, 0, 0, 445, 446, 447, 448, 449, 0, + 450, 451, 452, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 471, 0, 472, 0, 474, + 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2383, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, @@ -7674,88 +7669,137 @@ static const yytype_int16 yytable[] = 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, - 222, 0, 223, 0, 0, 0, 226, 227, 506, 0, - 230, 0, 231, 0, 232, 233, 234, 0, 235, 236, - 237, 238, 239, 240, 241, 0, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 0, 256, 257, 258, 259, 260, 261, 262, 263, 0, - 264, 0, 265, 0, 0, 268, 0, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 0, 278, 0, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 507, 289, + 222, 0, 223, 0, 0, 0, 226, 227, 509, 0, + 230, 0, 231, 0, 232, 233, 234, 235, 0, 236, + 237, 238, 239, 240, 241, 242, 0, 244, 245, 246, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 0, 257, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 266, 0, 0, 269, 0, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 0, 279, 0, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 0, 309, - 310, 311, 312, 313, 0, 314, 315, 0, 317, 0, - 318, 319, 320, 321, 322, 323, 0, 324, 325, 0, - 0, 326, 327, 328, 0, 0, 329, 330, 0, 332, - 0, 334, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 347, 0, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 364, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 0, - 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 0, 411, 412, - 413, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 508, 428, 429, 430, 0, - 431, 432, 0, 433, 0, 435, 436, 437, 438, 439, - 0, 440, 441, 0, 0, 442, 443, 444, 445, 446, - 0, 447, 448, 449, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 468, 0, 469, 0, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 300, 301, 302, 303, 304, 305, 306, 307, 308, 0, + 310, 311, 312, 313, 314, 0, 315, 316, 0, 318, + 0, 319, 320, 321, 322, 323, 324, 0, 325, 326, + 0, 0, 327, 328, 329, 0, 0, 330, 331, 332, + 0, 334, 0, 336, 337, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 0, 0, 0, 0, 347, 348, + 349, 0, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 364, 365, 366, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, 0, + 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 511, 430, 431, + 432, 0, 433, 434, 0, 435, 0, 437, 438, 439, + 440, 441, 0, 442, 443, 444, 0, 0, 445, 446, + 447, 448, 449, 0, 450, 451, 452, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 0, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 470, 471, + 0, 472, 0, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 505, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 3098, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, - 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, - 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, - 139, 140, 141, 0, 0, 142, 143, 144, 145, 146, - 0, 147, 148, 149, 150, 151, 0, 0, 0, 153, - 154, 155, 156, 157, 158, 0, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 0, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, - 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, - 0, 222, 0, 223, 0, 0, 0, 226, 227, 506, - 0, 230, 0, 231, 0, 232, 233, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 0, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 0, 256, 257, 258, 259, 260, 261, 262, 263, - 0, 264, 0, 265, 0, 0, 268, 0, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 0, 278, 0, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 507, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 0, - 309, 310, 311, 312, 313, 0, 314, 315, 0, 317, - 0, 318, 319, 320, 321, 322, 323, 0, 324, 325, - 0, 0, 326, 327, 328, 0, 0, 329, 330, 0, - 332, 0, 334, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 347, - 0, 349, 350, 351, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 364, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 0, 385, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 0, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, 0, 411, - 412, 413, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 508, 428, 429, 430, - 0, 431, 432, 0, 433, 0, 435, 436, 437, 438, - 439, 0, 440, 441, 0, 0, 442, 443, 444, 445, - 446, 0, 447, 448, 449, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 467, 468, 0, 469, - 0, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 505, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1989, 0, + 497, 498, 499, 500, 508, 0, 530, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 2491, 0, 0, 110, 111, 112, 113, 114, 115, 116, + 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, + 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, + 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, + 144, 145, 146, 0, 147, 148, 149, 150, 151, 0, + 0, 0, 153, 154, 155, 156, 157, 158, 0, 160, + 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, + 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, + 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, + 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, + 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, + 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, + 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, + 226, 227, 509, 0, 230, 0, 231, 0, 232, 233, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 0, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 0, 257, 258, 259, 260, + 261, 262, 263, 264, 0, 265, 0, 266, 0, 0, + 269, 0, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 0, 279, 0, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 0, 310, 311, 312, 313, 314, 0, + 315, 316, 0, 318, 0, 319, 320, 321, 322, 323, + 324, 0, 325, 326, 0, 0, 327, 328, 329, 0, + 0, 330, 331, 332, 0, 334, 0, 336, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 349, 0, 351, 352, 353, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 366, 367, 368, 369, 370, 0, 371, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 0, 413, 414, 415, 416, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, + 428, 511, 430, 431, 432, 0, 433, 434, 0, 435, + 0, 437, 438, 439, 440, 441, 0, 442, 443, 444, + 0, 0, 445, 446, 447, 448, 449, 0, 450, 451, + 452, 453, 454, 455, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 470, 471, 0, 472, 0, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 498, 499, 500, 508, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 3101, 0, 0, 110, 111, 112, + 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, + 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, + 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, + 0, 0, 142, 143, 144, 145, 146, 0, 147, 148, + 149, 150, 151, 0, 0, 0, 153, 154, 155, 156, + 157, 158, 0, 160, 161, 162, 0, 163, 164, 165, + 166, 167, 168, 0, 0, 170, 171, 172, 173, 174, + 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, + 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, + 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, + 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, + 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, + 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, + 223, 0, 0, 0, 226, 227, 509, 0, 230, 0, + 231, 0, 232, 233, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 0, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 0, + 257, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 266, 0, 0, 269, 0, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 0, 279, 0, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, + 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 0, 310, 311, + 312, 313, 314, 0, 315, 316, 0, 318, 0, 319, + 320, 321, 322, 323, 324, 0, 325, 326, 0, 0, + 327, 328, 329, 0, 0, 330, 331, 332, 0, 334, + 0, 336, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 349, 0, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 366, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 0, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 0, 413, 414, + 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 511, 430, 431, 432, 0, + 433, 434, 0, 435, 0, 437, 438, 439, 440, 441, + 0, 442, 443, 444, 0, 0, 445, 446, 447, 448, + 449, 0, 450, 451, 452, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 470, 471, 0, 472, + 0, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1992, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, @@ -7771,472 +7815,444 @@ static const yytype_int16 yytable[] = 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, 226, 227, - 506, 0, 230, 0, 231, 0, 232, 233, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 0, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 0, 256, 257, 258, 259, 260, 261, 262, - 263, 0, 264, 0, 265, 0, 0, 268, 0, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 0, 278, - 0, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 507, 289, 290, 291, 292, 293, 294, 295, 296, 297, + 509, 0, 230, 0, 231, 0, 232, 233, 234, 235, + 0, 236, 237, 238, 239, 240, 241, 242, 0, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 0, 257, 258, 259, 260, 261, 262, + 263, 264, 0, 265, 0, 266, 0, 0, 269, 0, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 0, + 279, 0, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 0, 309, 310, 311, 312, 313, 0, 314, 315, 0, - 317, 0, 318, 319, 320, 321, 322, 323, 0, 324, - 325, 0, 0, 326, 327, 328, 0, 0, 329, 330, - 0, 332, 0, 334, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 347, 0, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 364, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 0, 385, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 0, - 411, 412, 413, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 508, 428, 429, - 430, 0, 431, 432, 0, 433, 0, 435, 436, 437, - 438, 439, 0, 440, 441, 0, 0, 442, 443, 444, - 445, 446, 0, 447, 448, 449, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 468, 0, - 469, 0, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, + 308, 0, 310, 311, 312, 313, 314, 0, 315, 316, + 0, 318, 0, 319, 320, 321, 322, 323, 324, 0, + 325, 326, 0, 0, 327, 328, 329, 0, 0, 330, + 331, 332, 0, 334, 0, 336, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 349, 0, 351, 352, 353, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 366, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 0, 413, 414, 415, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 511, + 430, 431, 432, 0, 433, 434, 0, 435, 0, 437, + 438, 439, 440, 441, 0, 442, 443, 444, 0, 0, + 445, 446, 447, 448, 449, 0, 450, 451, 452, 453, + 454, 455, 456, 457, 0, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 470, 471, 0, 472, 0, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 505, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 2091, - 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, - 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, - 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, - 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, - 145, 146, 0, 147, 148, 149, 150, 151, 0, 0, - 0, 153, 154, 155, 156, 157, 158, 0, 160, 161, - 162, 0, 163, 164, 165, 166, 167, 168, 0, 0, - 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, - 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, - 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, - 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, - 220, 221, 0, 222, 0, 223, 0, 0, 0, 226, - 227, 506, 0, 230, 0, 231, 0, 232, 233, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 0, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 0, 256, 257, 258, 259, 260, 261, - 262, 263, 0, 264, 0, 265, 0, 0, 268, 0, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 0, - 278, 0, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 507, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 0, 309, 310, 311, 312, 313, 0, 314, 315, - 0, 317, 0, 318, 319, 320, 321, 322, 323, 0, - 324, 325, 0, 0, 326, 327, 328, 0, 0, 329, - 330, 0, 332, 0, 334, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 347, 0, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 364, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 0, 385, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 0, 411, 412, 413, 414, 0, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 508, 428, - 429, 430, 0, 431, 432, 0, 433, 0, 435, 436, - 437, 438, 439, 0, 440, 441, 0, 0, 442, 443, - 444, 445, 446, 0, 447, 448, 449, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 468, - 0, 469, 0, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 0, 2795, 1408, 629, 0, 0, - 1969, 996, 0, 0, 0, 0, 0, 1970, 1971, 0, - 2965, 1972, 1973, 1974, 110, 111, 112, 113, 114, 115, - 116, 117, 1184, 118, 119, 120, 1185, 1186, 1187, 2796, - 1188, 1189, 1190, 1191, 2797, 122, 123, 1192, 124, 125, - 126, 2798, 128, 129, 130, 0, 752, 2799, 754, 755, - 1193, 136, 137, 138, 139, 140, 141, 1194, 1195, 142, - 143, 756, 757, 146, 1196, 147, 148, 149, 150, 0, - 1197, 2800, 1198, 2801, 154, 155, 156, 157, 158, 2802, - 160, 161, 162, 1199, 163, 164, 165, 166, 167, 168, - 1200, 2803, 170, 171, 172, 173, 174, 175, 176, 177, - 178, 179, 762, 763, 182, 1201, 183, 1202, 184, 185, - 186, 187, 188, 189, 1203, 190, 191, 192, 193, 194, - 1204, 1205, 195, 196, 764, 198, 199, 1206, 200, 201, - 202, 1207, 203, 204, 205, 1208, 206, 207, 208, 209, - 0, 211, 212, 213, 214, 215, 0, 1209, 217, 1210, - 218, 219, 765, 221, 1211, 222, 1212, 223, 2804, 1213, - 2805, 226, 227, 2806, 2807, 230, 1214, 231, 1215, 0, - 0, 234, 1216, 235, 236, 237, 238, 239, 240, 241, - 2808, 243, 244, 245, 246, 1217, 247, 248, 249, 250, - 251, 252, 253, 1218, 254, 2809, 0, 257, 258, 259, - 260, 261, 772, 773, 1219, 774, 1220, 265, 2810, 2811, - 268, 2812, 270, 271, 272, 273, 274, 275, 1221, 1222, - 276, 2813, 278, 2814, 1223, 280, 281, 282, 283, 284, - 285, 286, 287, 2815, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 781, 2816, 783, 310, 311, 312, 2817, 1224, - 314, 315, 2818, 317, 1225, 0, 319, 785, 321, 322, - 323, 1226, 324, 325, 1227, 1228, 2819, 327, 328, 1229, - 1230, 329, 0, 2820, 332, 2821, 0, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 1231, 1232, 1233, - 1234, 345, 346, 0, 2822, 349, 350, 0, 352, 353, - 354, 1235, 355, 356, 357, 358, 359, 360, 1236, 361, - 362, 363, 789, 365, 366, 367, 368, 1237, 369, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 1238, 382, 383, 2823, 385, 386, 387, 791, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 1239, 2824, 400, 401, 402, 403, 404, 405, 2825, 407, - 408, 409, 2826, 411, 412, 795, 414, 1240, 415, 416, - 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 2827, 428, 0, 430, 1241, 431, 432, 1242, 433, 2828, - 435, 436, 437, 438, 439, 1243, 798, 799, 1244, 1245, - 442, 443, 0, 445, 0, 1246, 447, 448, 2829, 450, - 451, 452, 453, 454, 2830, 1248, 455, 456, 457, 1249, - 458, 459, 460, 461, 1250, 462, 463, 464, 465, 466, - 0, 802, 1252, 469, 2831, 471, 472, 473, 474, 475, - 1253, 1254, 476, 1255, 1256, 477, 478, 479, 480, 481, - 482, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 494, 495, 496, 497, 0, 505, 0, 1975, - 1976, 1977, 1969, 2832, 2833, 1980, 1981, 1982, 1983, 1970, - 1971, 0, 0, 1972, 1973, 1974, 110, 111, 112, 113, - 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, - 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, - 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, - 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, - 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, - 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, - 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, - 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, - 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, - 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, - 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, - 0, 0, 0, 226, 227, 506, 0, 230, 0, 231, - 0, 232, 233, 234, 0, 235, 236, 237, 238, 239, - 240, 241, 0, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 0, 256, 257, - 258, 259, 260, 261, 262, 263, 0, 264, 0, 265, - 0, 0, 268, 0, 270, 271, 272, 273, 274, 275, - 0, 0, 276, 0, 278, 0, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 507, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 0, 309, 310, 311, 312, - 313, 0, 314, 315, 0, 317, 0, 318, 319, 320, - 321, 322, 323, 0, 324, 325, 0, 0, 326, 327, - 328, 0, 0, 329, 330, 0, 332, 0, 334, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 347, 0, 349, 350, 351, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 364, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 0, 385, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 0, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 0, 411, 412, 413, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 508, 428, 429, 430, 0, 431, 432, 0, - 433, 0, 435, 436, 437, 438, 439, 0, 440, 441, - 0, 0, 442, 443, 444, 445, 446, 0, 447, 448, - 449, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 467, 468, 0, 469, 0, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 0, 0, - 0, 1975, 1976, 1977, 0, 1978, 1979, 1980, 1981, 1982, - 1983, 1560, 0, 0, 1561, 0, 0, 0, 1562, 1563, - 1564, 1565, 1566, 1567, 1568, 0, 0, 0, 0, 0, - 0, 0, 0, 1560, 0, 0, 1561, 0, 0, 1569, - 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 1571, - 0, 0, 0, 0, 0, 0, 1572, 0, 0, 0, - 0, 1569, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1571, 0, 0, 0, 0, 0, 0, 1572, 0, - 1560, 1573, 0, 1561, 0, 0, 0, 1562, 1563, 1564, - 1565, 1566, 1567, 1568, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1573, 0, 0, 0, 0, 1569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1571, 0, - 0, 0, 0, 0, 0, 1572, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1560, 0, 0, 1561, 0, - 0, 0, 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, - 1573, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1571, 0, 1574, 0, 0, 0, 0, - 1572, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1575, 0, 0, 0, 0, 1576, 1574, 0, 0, - 0, 0, 0, 0, 0, 1573, 0, 0, 0, 0, - 0, 0, 0, 1575, 0, 0, 0, 0, 1576, 1577, - 1578, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1579, 0, 0, 0, 0, 0, - 0, 1577, 1578, 0, 1574, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1579, 0, 0, 0, - 1575, 0, 0, 0, 0, 1576, 0, 0, 0, 0, - 1580, 0, 0, 1581, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1582, 1577, 1578, - 1583, 0, 1580, 0, 0, 1581, 0, 0, 0, 1574, - 0, 0, 0, 1579, 0, 0, 0, 0, 0, 1582, - 0, 0, 1583, 0, 0, 1575, 0, 0, 0, 0, - 1576, 0, 0, 0, 0, 1560, 0, 0, 1561, 0, - 0, 0, 1562, 1563, 1564, 1565, 1566, 1567, 1568, 1580, - 0, 0, 1581, 1577, 1578, 0, 0, 0, 0, 0, - 0, 0, 0, 1569, 0, 0, 1582, 0, 1579, 1583, - 0, 0, 0, 1571, 0, 0, 0, 0, 0, 0, - 1572, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1584, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1580, 1573, 0, 1581, 0, 0, - 0, 0, 0, 0, 0, 1584, 0, 0, 0, 0, - 0, 1582, 0, 0, 1583, 0, 0, 0, 0, 0, + 495, 496, 497, 498, 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 2094, 0, 0, 110, 111, 112, 113, 114, + 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, + 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, + 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, + 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, + 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, + 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, + 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, + 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, + 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, + 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, + 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, + 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, + 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, + 0, 0, 226, 227, 509, 0, 230, 0, 231, 0, + 232, 233, 234, 235, 0, 236, 237, 238, 239, 240, + 241, 242, 0, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 0, 257, 258, + 259, 260, 261, 262, 263, 264, 0, 265, 0, 266, + 0, 0, 269, 0, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 0, 279, 0, 0, 281, 282, 283, + 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 0, 310, 311, 312, 313, + 314, 0, 315, 316, 0, 318, 0, 319, 320, 321, + 322, 323, 324, 0, 325, 326, 0, 0, 327, 328, + 329, 0, 0, 330, 331, 332, 0, 334, 0, 336, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 349, 0, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 366, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 0, 413, 414, 415, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 511, 430, 431, 432, 0, 433, 434, + 0, 435, 0, 437, 438, 439, 440, 441, 0, 442, + 443, 444, 0, 0, 445, 446, 447, 448, 449, 0, + 450, 451, 452, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 471, 0, 472, 0, 474, + 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 0, 2798, 1411, 632, 0, 0, 1972, 999, 0, 0, + 0, 0, 0, 1973, 1974, 0, 2968, 1975, 1976, 1977, + 110, 111, 112, 113, 114, 115, 116, 117, 1187, 118, + 119, 120, 1188, 1189, 1190, 2799, 1191, 1192, 1193, 1194, + 2800, 122, 123, 1195, 124, 125, 126, 2801, 128, 129, + 130, 0, 755, 2802, 757, 758, 1196, 136, 137, 138, + 139, 140, 141, 1197, 1198, 142, 143, 759, 760, 146, + 1199, 147, 148, 149, 150, 0, 1200, 2803, 1201, 2804, + 154, 155, 156, 157, 158, 2805, 160, 161, 162, 1202, + 163, 164, 165, 166, 167, 168, 1203, 2806, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 765, 766, + 182, 1204, 183, 1205, 184, 185, 186, 187, 188, 189, + 1206, 190, 191, 192, 193, 194, 1207, 1208, 195, 196, + 767, 198, 199, 1209, 200, 201, 202, 1210, 203, 204, + 205, 1211, 206, 207, 208, 209, 0, 211, 212, 213, + 214, 215, 0, 1212, 217, 1213, 218, 219, 768, 221, + 1214, 222, 1215, 223, 2807, 1216, 2808, 226, 227, 2809, + 2810, 230, 1217, 231, 1218, 0, 0, 234, 235, 1219, + 236, 237, 238, 239, 240, 241, 242, 2811, 244, 245, + 246, 247, 1220, 248, 249, 250, 251, 252, 253, 254, + 1221, 255, 2812, 0, 258, 259, 260, 261, 262, 775, + 776, 1222, 777, 1223, 266, 2813, 2814, 269, 2815, 271, + 272, 273, 274, 275, 276, 1224, 1225, 277, 2816, 279, + 2817, 1226, 281, 282, 283, 284, 285, 286, 287, 288, + 2818, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 784, + 2819, 786, 311, 312, 313, 2820, 1227, 315, 316, 2821, + 318, 1228, 0, 320, 788, 322, 323, 324, 1229, 325, + 326, 1230, 1231, 2822, 328, 329, 1232, 1233, 330, 331, + 0, 2823, 334, 2824, 0, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 1234, 1235, 1236, 1237, 347, + 348, 0, 2825, 351, 352, 0, 354, 355, 356, 1238, + 357, 358, 359, 360, 361, 362, 1239, 363, 364, 365, + 792, 367, 368, 369, 370, 1240, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 1241, + 384, 385, 2826, 387, 388, 389, 794, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 1242, 2827, + 402, 403, 404, 405, 406, 407, 2828, 409, 410, 411, + 2829, 413, 414, 798, 416, 1243, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 2830, 430, + 0, 432, 1244, 433, 434, 1245, 435, 2831, 437, 438, + 439, 440, 441, 1246, 442, 801, 802, 1247, 1248, 445, + 446, 0, 448, 0, 1249, 450, 451, 2832, 453, 454, + 455, 456, 457, 2833, 1251, 458, 459, 460, 1252, 461, + 462, 463, 464, 1253, 465, 466, 467, 468, 469, 0, + 805, 1255, 472, 2834, 474, 475, 476, 477, 478, 1256, + 1257, 479, 1258, 1259, 480, 481, 482, 483, 484, 485, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 497, 498, 499, 500, 0, 508, 0, 1978, 1979, + 1980, 1972, 2835, 2836, 1983, 1984, 1985, 1986, 1973, 1974, + 0, 0, 1975, 1976, 1977, 110, 111, 112, 113, 114, + 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, + 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, + 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, + 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, + 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, + 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, + 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, + 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, + 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, + 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, + 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, + 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, + 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, + 0, 0, 226, 227, 509, 0, 230, 0, 231, 0, + 232, 233, 234, 235, 0, 236, 237, 238, 239, 240, + 241, 242, 0, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 0, 257, 258, + 259, 260, 261, 262, 263, 264, 0, 265, 0, 266, + 0, 0, 269, 0, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 0, 279, 0, 0, 281, 282, 283, + 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 0, 310, 311, 312, 313, + 314, 0, 315, 316, 0, 318, 0, 319, 320, 321, + 322, 323, 324, 0, 325, 326, 0, 0, 327, 328, + 329, 0, 0, 330, 331, 332, 0, 334, 0, 336, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 349, 0, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 366, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 0, 413, 414, 415, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 511, 430, 431, 432, 0, 433, 434, + 0, 435, 0, 437, 438, 439, 440, 441, 0, 442, + 443, 444, 0, 0, 445, 446, 447, 448, 449, 0, + 450, 451, 452, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 471, 0, 472, 0, 474, + 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 0, 0, 0, 1978, 1979, 1980, 0, 1981, 1982, 1983, + 1984, 1985, 1986, 1563, 0, 0, 1564, 0, 0, 0, + 1565, 1566, 1567, 1568, 1569, 1570, 1571, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, + 0, 1572, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1574, 0, 1563, 0, 0, 1564, 0, 1575, 0, + 1565, 1566, 1567, 1568, 1569, 1570, 1571, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1572, 0, 1576, 0, 0, 0, 0, 0, 0, + 0, 1574, 0, 1563, 0, 0, 1564, 0, 1575, 0, + 1565, 1566, 1567, 1568, 1569, 1570, 1571, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1585, 0, 1574, - 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, 1594, - 0, 0, 0, 0, 2414, 1575, 0, 1584, 0, 1585, - 1576, 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, - 1593, 1594, 0, 0, 0, 0, 2622, 0, 0, 0, - 0, 0, 0, 1577, 1578, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1579, 0, - 0, 0, 0, 0, 0, 0, 1585, 0, 0, 1586, - 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, 1594, 0, - 0, 0, 0, 2674, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1580, 0, 0, 1581, 0, 0, + 0, 1572, 0, 1576, 0, 0, 0, 0, 0, 0, + 0, 1574, 0, 1563, 0, 0, 1564, 0, 1575, 0, + 1565, 1566, 1567, 1568, 1569, 1570, 1571, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1582, 0, 0, 1583, 0, 0, 0, 0, 0, - 0, 1585, 0, 0, 1586, 1587, 1588, 0, 1589, 1590, - 1591, 1592, 1593, 1594, 0, 0, 0, 0, 2741, 0, - 0, 0, 0, 0, 1560, 0, 0, 1561, 0, 0, - 0, 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, - 0, 0, 0, 0, 0, 0, 1560, 0, 0, 1561, - 0, 0, 1569, 1562, 1563, 1564, 1565, 1566, 1567, 1568, - 0, 0, 1571, 0, 0, 0, 0, 0, 0, 1572, - 0, 0, 0, 0, 1569, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1571, 0, 0, 1584, 0, 0, - 0, 1572, 0, 1560, 1573, 0, 1561, 0, 0, 0, - 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1573, 0, 0, 0, - 0, 1569, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1571, 0, 0, 0, 0, 0, 0, 1572, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1560, 0, - 0, 1561, 0, 0, 0, 1562, 1563, 1564, 1565, 1566, - 1567, 1568, 0, 1573, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1569, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1571, 0, 1574, 0, - 0, 1585, 0, 1572, 1586, 1587, 1588, 0, 1589, 1590, - 1591, 1592, 1593, 1594, 1575, 0, 0, 0, 2957, 1576, - 1574, 0, 0, 0, 0, 0, 0, 0, 1573, 0, - 0, 0, 0, 0, 0, 0, 1575, 0, 0, 0, - 0, 1576, 1577, 1578, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1579, 0, 0, - 0, 0, 0, 0, 1577, 1578, 0, 1574, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1579, - 0, 0, 0, 1575, 0, 0, 0, 0, 1576, 0, - 0, 0, 0, 1580, 0, 0, 1581, 0, 0, 0, + 0, 1572, 0, 1576, 0, 0, 0, 0, 0, 0, + 0, 1574, 0, 1563, 0, 0, 1564, 1577, 1575, 0, + 1565, 1566, 1567, 1568, 1569, 1570, 1571, 0, 0, 0, + 0, 0, 0, 0, 1578, 0, 0, 0, 0, 1579, + 0, 1572, 0, 1576, 0, 0, 0, 0, 0, 0, + 0, 1574, 0, 0, 0, 0, 0, 1577, 1575, 0, + 0, 0, 1580, 1581, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1578, 0, 0, 1582, 0, 1579, + 0, 0, 0, 1576, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1577, 0, 0, + 0, 0, 1580, 1581, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1583, 1578, 0, 1584, 1582, 0, 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1582, 1577, 1578, 1583, 0, 1580, 0, 0, 1581, 0, - 0, 0, 1574, 0, 0, 0, 1579, 0, 0, 0, - 0, 0, 1582, 0, 0, 1583, 0, 0, 1575, 0, - 0, 0, 0, 1576, 0, 0, 0, 0, 1560, 0, - 0, 1561, 0, 0, 0, 1562, 1563, 1564, 1565, 1566, - 1567, 1568, 1580, 0, 0, 1581, 1577, 1578, 0, 0, - 0, 0, 0, 0, 0, 0, 1569, 0, 0, 1582, - 0, 1579, 1583, 0, 0, 0, 1571, 0, 0, 0, - 0, 0, 0, 1572, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1584, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1580, 1573, 0, - 1581, 0, 0, 0, 0, 0, 0, 0, 1584, 0, - 0, 0, 0, 0, 1582, 0, 0, 1583, 0, 0, + 1585, 0, 0, 1586, 0, 0, 0, 1577, 0, 0, + 0, 0, 1580, 1581, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1583, 1578, 0, 1584, 1582, 0, 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1585, 0, 0, 1586, 0, 0, 0, 1577, 0, 0, + 0, 0, 1580, 1581, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1583, 1578, 0, 1584, 1582, 0, 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, + 0, 0, 1580, 1581, 0, 0, 0, 1587, 0, 0, + 0, 0, 0, 1583, 0, 0, 1584, 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1584, 0, 0, 0, 0, + 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1587, 0, 0, + 0, 0, 0, 1583, 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1585, 0, 1574, 1586, 1587, 1588, 0, 1589, 1590, 1591, - 1592, 1593, 1594, 0, 0, 0, 0, 2964, 1575, 0, - 1584, 0, 1585, 1576, 0, 1586, 1587, 1588, 0, 1589, - 1590, 1591, 1592, 1593, 1594, 0, 0, 0, 0, 3117, - 0, 0, 0, 0, 0, 0, 1577, 1578, 0, 0, + 0, 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, + 1593, 1594, 1595, 1596, 1597, 0, 0, 1587, 0, 2315, + 0, 0, 0, 0, 0, 0, 0, 1563, 0, 0, + 1564, 0, 0, 0, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, + 1593, 1594, 1595, 1596, 1597, 1572, 0, 1587, 0, 2400, + 0, 0, 0, 0, 0, 1574, 0, 1563, 0, 0, + 1564, 0, 1575, 0, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, + 1593, 1594, 1595, 1596, 1597, 1572, 0, 1576, 0, 2417, + 0, 0, 0, 0, 0, 1574, 0, 1563, 0, 0, + 1564, 0, 1575, 0, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, + 1593, 1594, 1595, 1596, 1597, 1572, 0, 1576, 0, 2625, + 0, 0, 0, 0, 0, 1574, 0, 1563, 0, 0, + 1564, 0, 1575, 0, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, + 1593, 1594, 1595, 1596, 1597, 1572, 0, 1576, 0, 2677, + 0, 0, 0, 0, 0, 1574, 0, 1563, 0, 0, + 1564, 1577, 1575, 0, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 0, 0, 0, 0, 0, 0, 0, 1578, 0, + 0, 0, 0, 1579, 0, 1572, 0, 1576, 0, 0, + 0, 0, 0, 0, 0, 1574, 0, 0, 0, 0, + 0, 1577, 1575, 0, 0, 0, 1580, 1581, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1578, 0, + 0, 1582, 0, 1579, 0, 0, 0, 1576, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1579, 0, 0, 0, 0, 0, 0, 0, 1585, - 0, 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, - 1593, 1594, 0, 0, 0, 0, 3139, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1580, 0, 0, - 1581, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1582, 0, 0, 1583, 0, 0, - 0, 0, 0, 0, 1585, 0, 0, 1586, 1587, 1588, - 0, 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, 0, - 0, 3239, 0, 0, 0, 0, 0, 1560, 0, 0, - 1561, 0, 0, 0, 1562, 1563, 1564, 1565, 1566, 1567, - 1568, 0, 0, 0, 0, 0, 0, 0, 0, 1560, - 0, 0, 1561, 0, 0, 1569, 1562, 1563, 1564, 1565, - 1566, 1567, 1568, 0, 0, 1571, 0, 0, 0, 0, - 0, 0, 1572, 0, 0, 0, 0, 1569, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1571, 0, 0, - 1584, 0, 0, 0, 1572, 0, 1560, 1573, 0, 1561, - 0, 0, 0, 1562, 1563, 1564, 1565, 1566, 1567, 1568, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1573, - 0, 0, 0, 0, 1569, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1571, 0, 0, 0, 0, 0, - 0, 1572, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1560, 0, 0, 1561, 0, 0, 0, 1562, 1563, - 1564, 1565, 1566, 1567, 1568, 0, 1573, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1569, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1571, - 0, 1574, 0, 0, 1585, 0, 1572, 1586, 1587, 1588, - 0, 1589, 1590, 1591, 1592, 1593, 1594, 1575, 0, 0, - 0, 3296, 1576, 1574, 0, 0, 0, 0, 0, 0, - 0, 1573, 0, 0, 0, 0, 0, 0, 0, 1575, - 0, 0, 0, 0, 1576, 1577, 1578, 0, 0, 0, + 0, 1577, 0, 0, 0, 0, 1580, 1581, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1583, 1578, 0, + 1584, 1582, 0, 1579, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1585, 0, 0, 1586, 0, 0, + 0, 1577, 0, 0, 0, 0, 1580, 1581, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1583, 1578, 0, + 1584, 1582, 0, 1579, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1585, 0, 0, 1586, 0, 0, + 0, 1577, 0, 0, 0, 0, 1580, 1581, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1583, 1578, 0, + 1584, 1582, 0, 1579, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1585, 0, 0, 1586, 0, 0, + 0, 0, 0, 0, 0, 0, 1580, 1581, 0, 0, + 0, 1587, 0, 0, 0, 0, 0, 1583, 0, 0, + 1584, 1582, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1579, 0, 0, 0, 0, 0, 0, 1577, 1578, 0, - 1574, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1579, 0, 0, 0, 1575, 0, 0, 0, - 0, 1576, 0, 0, 0, 0, 1580, 0, 0, 1581, + 0, 1587, 0, 0, 0, 0, 0, 1583, 0, 0, + 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1582, 1577, 1578, 1583, 0, 1580, 0, - 0, 1581, 0, 0, 0, 1574, 0, 0, 0, 1579, - 0, 0, 0, 0, 0, 1582, 0, 0, 1583, 0, - 0, 1575, 0, 0, 0, 0, 1576, 0, 0, 0, + 0, 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1580, 0, 0, 1581, 1577, - 1578, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1582, 0, 1579, 1583, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1588, 0, 0, 1589, + 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 0, + 0, 1587, 0, 2744, 0, 0, 0, 0, 0, 0, + 0, 1563, 0, 0, 1564, 0, 0, 0, 1565, 1566, + 1567, 1568, 1569, 1570, 1571, 0, 1588, 0, 0, 1589, + 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 1572, + 0, 1587, 0, 2960, 0, 0, 0, 0, 0, 1574, + 0, 1563, 0, 0, 1564, 0, 1575, 0, 1565, 1566, + 1567, 1568, 1569, 1570, 1571, 0, 1588, 0, 0, 1589, + 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 1572, + 0, 1576, 0, 2967, 0, 0, 0, 0, 0, 1574, + 0, 1563, 0, 0, 1564, 0, 1575, 0, 1565, 1566, + 1567, 1568, 1569, 1570, 1571, 0, 1588, 0, 0, 1589, + 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 1572, + 0, 1576, 0, 3120, 0, 0, 0, 0, 0, 1574, + 0, 1563, 0, 0, 1564, 0, 1575, 0, 1565, 1566, + 1567, 1568, 1569, 1570, 1571, 0, 1588, 0, 0, 1589, + 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 1572, + 0, 1576, 0, 3142, 0, 0, 0, 0, 0, 1574, + 0, 1563, 0, 0, 1564, 1577, 1575, 0, 1565, 1566, + 1567, 1568, 1569, 1570, 1571, 0, 0, 0, 0, 0, + 0, 0, 1578, 0, 0, 0, 0, 1579, 0, 1572, + 0, 1576, 0, 0, 0, 0, 0, 0, 0, 1574, + 0, 0, 0, 0, 0, 1577, 1575, 0, 0, 0, + 1580, 1581, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1578, 0, 0, 1582, 0, 1579, 0, 0, + 0, 1576, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1577, 0, 0, 0, 0, + 1580, 1581, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1583, 1578, 0, 1584, 1582, 0, 1579, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1585, 0, + 0, 1586, 0, 0, 0, 1577, 0, 0, 0, 0, + 1580, 1581, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1583, 1578, 0, 1584, 1582, 0, 1579, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1585, 0, + 0, 1586, 0, 0, 0, 1577, 0, 0, 0, 0, + 1580, 1581, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1583, 1578, 0, 1584, 1582, 0, 1579, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1585, 0, + 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, + 1580, 1581, 0, 0, 0, 1587, 0, 0, 0, 0, + 0, 1583, 0, 0, 1584, 1582, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1585, 0, + 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1587, 0, 0, 0, 0, + 0, 1583, 0, 0, 1584, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1585, 0, + 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1580, 0, 0, 1581, 0, 0, 0, 0, 0, 0, - 0, 1584, 0, 0, 0, 0, 0, 1582, 0, 0, - 1583, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1588, 0, 0, 1589, 1590, 1591, 0, 1592, 1593, 1594, + 1595, 1596, 1597, 0, 0, 1587, 0, 3242, 0, 0, + 0, 0, 0, 0, 0, 1563, 0, 0, 1564, 0, + 0, 0, 1565, 1566, 1567, 1568, 1569, 1570, 1571, 0, + 1588, 0, 0, 1589, 1590, 1591, 0, 1592, 1593, 1594, + 1595, 1596, 1597, 1572, 0, 1587, 0, 3299, 0, 0, + 0, 0, 0, 1574, 0, 0, 1563, 0, 0, 1564, + 1575, 0, 0, 1565, 1566, 1567, 1568, 1569, 1570, 1571, + 1588, 0, 0, 1589, 1590, 1591, 0, 1592, 1593, 1594, + 1595, 1596, 1597, 0, 1572, 1576, 0, 3321, 0, 0, + 0, 0, 0, 0, 1574, 0, 0, 0, 0, 0, + 0, 1575, 0, 0, 0, 0, 0, 0, 0, 0, + 1588, 0, 0, 1589, 1590, 1591, 0, 1592, 1593, 1594, + 1595, 1596, 1597, 0, 0, 1748, 1576, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1584, 0, + 1588, 0, 0, 1589, 1590, 1591, 0, 1592, 1593, 1594, + 1595, 1596, 1597, 0, 0, 2698, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1577, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1578, 0, 0, 0, + 0, 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1577, 0, 0, 0, 1580, 1581, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1578, 0, 1582, + 0, 0, 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1585, 0, 0, 1586, 1587, 1588, 0, - 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, 0, 0, - 3318, 0, 0, 1584, 0, 1585, 0, 0, 1586, 1587, - 1588, 0, 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, - 2695, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1580, 1581, 0, 0, 0, + 0, 0, 0, 0, 0, 1583, 0, 0, 1584, 0, + 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1583, 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1585, 0, 0, 1586, 1587, 1588, 0, 1589, - 1590, 1591, 1592, 1593, 1594, 0, 0, 3107, 0, 0, + 0, 0, 0, 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1183, 0, 0, 1585, 0, 0, - 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, 1594, - 0, 0, 3280, 110, 111, 112, 113, 114, 115, 116, - 117, 1184, 118, 119, 120, 1185, 1186, 1187, 835, 1188, - 1189, 1190, 1191, 836, 122, 123, 1192, 124, 125, 126, - 837, 128, 129, 130, 838, 839, 840, 841, 842, 1193, - 136, 137, 138, 139, 140, 141, 1194, 1195, 142, 143, - 843, 844, 146, 1196, 147, 148, 149, 150, 845, 1197, - 846, 1198, 847, 154, 155, 156, 157, 158, 848, 160, - 161, 162, 1199, 163, 164, 165, 166, 167, 168, 1200, - 849, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 850, 851, 182, 1201, 183, 1202, 184, 185, 186, - 187, 188, 189, 1203, 190, 191, 192, 193, 194, 1204, - 1205, 195, 196, 197, 198, 199, 1206, 200, 201, 202, - 1207, 203, 204, 205, 1208, 206, 207, 208, 209, 852, - 211, 212, 213, 214, 215, 853, 1209, 217, 1210, 218, - 219, 854, 221, 1211, 222, 1212, 223, 855, 1213, 856, - 226, 227, 857, 858, 230, 1214, 231, 1215, 859, 860, - 234, 1216, 235, 236, 237, 238, 239, 240, 241, 861, - 243, 244, 245, 246, 1217, 247, 248, 249, 250, 251, - 252, 253, 1218, 254, 862, 863, 257, 258, 259, 260, - 261, 864, 865, 1219, 866, 1220, 265, 867, 868, 268, - 869, 270, 271, 272, 273, 274, 275, 1221, 1222, 276, - 870, 278, 871, 1223, 280, 281, 282, 283, 284, 285, - 286, 287, 872, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 873, 874, 875, 310, 311, 312, 876, 1224, 314, - 315, 877, 317, 1225, 878, 319, 879, 321, 322, 323, - 1226, 324, 325, 1227, 1228, 326, 327, 328, 1229, 1230, - 329, 880, 881, 332, 882, 883, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 1231, 1232, 1233, 1234, - 345, 346, 884, 885, 349, 350, 886, 352, 353, 354, - 1235, 355, 356, 357, 358, 359, 360, 1236, 361, 362, - 363, 887, 365, 366, 367, 368, 1237, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 1238, 382, 383, 888, 385, 386, 387, 889, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 1239, - 890, 400, 401, 402, 403, 404, 405, 891, 407, 408, - 409, 892, 411, 412, 893, 414, 1240, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 894, - 428, 895, 430, 1241, 431, 432, 1242, 433, 896, 435, - 436, 437, 438, 439, 1243, 897, 898, 1244, 1245, 442, - 443, 899, 445, 900, 1246, 447, 448, 901, 450, 451, - 452, 453, 454, 1247, 1248, 455, 456, 457, 1249, 458, - 459, 460, 461, 1250, 462, 463, 464, 465, 466, 1251, - 903, 1252, 469, 904, 471, 472, 473, 474, 475, 1253, - 1254, 476, 1255, 1256, 477, 478, 479, 480, 481, 482, - 905, 906, 907, 908, 909, 910, 911, 912, 913, 914, - 915, 494, 495, 496, 497, 505, 0, 0, 0, 0, - 0, 0, 0, 0, 2000, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, - 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, - 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, - 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, - 143, 144, 145, 146, 0, 147, 148, 149, 150, 151, - 0, 0, 0, 153, 154, 155, 156, 157, 158, 0, - 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, - 0, 0, 170, 171, 172, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, - 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, - 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, - 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, - 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, - 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, - 0, 226, 227, 506, 0, 230, 0, 231, 0, 232, - 233, 234, 0, 235, 236, 237, 238, 239, 240, 241, - 0, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 0, 256, 257, 258, 259, - 260, 261, 262, 263, 0, 264, 0, 265, 0, 0, - 268, 0, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 0, 278, 0, 0, 280, 281, 282, 283, 284, - 285, 286, 287, 507, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 0, 309, 310, 311, 312, 313, 0, - 314, 315, 0, 317, 0, 318, 319, 320, 321, 322, - 323, 0, 324, 325, 0, 0, 326, 327, 328, 0, - 0, 329, 330, 0, 332, 0, 334, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 0, 0, 0, - 0, 345, 346, 347, 0, 349, 350, 351, 352, 353, - 354, 0, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 364, 365, 366, 367, 368, 0, 369, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 0, 385, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 0, 0, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, 0, 411, 412, 413, 414, 0, 415, 416, - 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 508, 428, 429, 430, 0, 431, 432, 0, 433, 0, - 435, 436, 437, 438, 439, 0, 440, 441, 0, 0, - 442, 443, 444, 445, 446, 0, 447, 448, 449, 450, - 451, 452, 453, 454, 0, 0, 455, 456, 457, 0, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 467, 468, 0, 469, 0, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 0, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 505, 0, 0, 0, - 0, 0, 0, 0, 0, 2615, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, - 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, - 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, - 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, - 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1587, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1588, 0, 0, 1589, 1590, 1591, + 0, 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, 3110, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1186, 0, 0, 1588, 0, 0, 1589, 1590, + 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, + 3283, 110, 111, 112, 113, 114, 115, 116, 117, 1187, + 118, 119, 120, 1188, 1189, 1190, 838, 1191, 1192, 1193, + 1194, 839, 122, 123, 1195, 124, 125, 126, 840, 128, + 129, 130, 841, 842, 843, 844, 845, 1196, 136, 137, + 138, 139, 140, 141, 1197, 1198, 142, 143, 846, 847, + 146, 1199, 147, 148, 149, 150, 848, 1200, 849, 1201, + 850, 154, 155, 156, 157, 158, 851, 160, 161, 162, + 1202, 163, 164, 165, 166, 167, 168, 1203, 852, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 853, + 854, 182, 1204, 183, 1205, 184, 185, 186, 187, 188, + 189, 1206, 190, 191, 192, 193, 194, 1207, 1208, 195, + 196, 197, 198, 199, 1209, 200, 201, 202, 1210, 203, + 204, 205, 1211, 206, 207, 208, 209, 855, 211, 212, + 213, 214, 215, 856, 1212, 217, 1213, 218, 219, 857, + 221, 1214, 222, 1215, 223, 858, 1216, 859, 226, 227, + 860, 861, 230, 1217, 231, 1218, 862, 863, 234, 235, + 1219, 236, 237, 238, 239, 240, 241, 242, 864, 244, + 245, 246, 247, 1220, 248, 249, 250, 251, 252, 253, + 254, 1221, 255, 865, 866, 258, 259, 260, 261, 262, + 867, 868, 1222, 869, 1223, 266, 870, 871, 269, 872, + 271, 272, 273, 274, 275, 276, 1224, 1225, 277, 873, + 279, 874, 1226, 281, 282, 283, 284, 285, 286, 287, + 288, 875, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 876, 877, 878, 311, 312, 313, 879, 1227, 315, 316, + 880, 318, 1228, 881, 320, 882, 322, 323, 324, 1229, + 325, 326, 1230, 1231, 327, 328, 329, 1232, 1233, 330, + 331, 883, 884, 334, 885, 886, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 1234, 1235, 1236, 1237, + 347, 348, 887, 888, 351, 352, 889, 354, 355, 356, + 1238, 357, 358, 359, 360, 361, 362, 1239, 363, 364, + 365, 890, 367, 368, 369, 370, 1240, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 1241, 384, 385, 891, 387, 388, 389, 892, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 1242, + 893, 402, 403, 404, 405, 406, 407, 894, 409, 410, + 411, 895, 413, 414, 896, 416, 1243, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 897, + 430, 898, 432, 1244, 433, 434, 1245, 435, 899, 437, + 438, 439, 440, 441, 1246, 442, 900, 901, 1247, 1248, + 445, 446, 902, 448, 903, 1249, 450, 451, 904, 453, + 454, 455, 456, 457, 1250, 1251, 458, 459, 460, 1252, + 461, 462, 463, 464, 1253, 465, 466, 467, 468, 469, + 1254, 906, 1255, 472, 907, 474, 475, 476, 477, 478, + 1256, 1257, 479, 1258, 1259, 480, 481, 482, 483, 484, + 485, 908, 909, 910, 911, 912, 913, 914, 915, 916, + 917, 918, 497, 498, 499, 500, 508, 0, 0, 0, + 0, 0, 0, 0, 0, 2003, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, + 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, + 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, + 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, + 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, + 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, @@ -8245,656 +8261,648 @@ static const yytype_int16 yytable[] = 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, - 0, 0, 226, 227, 506, 0, 230, 0, 231, 0, - 232, 233, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 0, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 0, 256, 257, 258, - 259, 260, 261, 262, 263, 0, 264, 0, 265, 0, - 0, 268, 0, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 0, 278, 0, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 507, 289, 290, 291, 292, 293, + 0, 0, 226, 227, 509, 0, 230, 0, 231, 0, + 232, 233, 234, 235, 0, 236, 237, 238, 239, 240, + 241, 242, 0, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 0, 257, 258, + 259, 260, 261, 262, 263, 264, 0, 265, 0, 266, + 0, 0, 269, 0, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 0, 279, 0, 0, 281, 282, 283, + 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 0, 309, 310, 311, 312, 313, - 0, 314, 315, 0, 317, 0, 318, 319, 320, 321, - 322, 323, 0, 324, 325, 0, 0, 326, 327, 328, - 0, 0, 329, 330, 0, 332, 0, 334, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 347, 0, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 364, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 0, 385, 386, 387, 388, + 304, 305, 306, 307, 308, 0, 310, 311, 312, 313, + 314, 0, 315, 316, 0, 318, 0, 319, 320, 321, + 322, 323, 324, 0, 325, 326, 0, 0, 327, 328, + 329, 0, 0, 330, 331, 332, 0, 334, 0, 336, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 349, 0, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 366, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 0, 411, 412, 413, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 508, 428, 429, 430, 0, 431, 432, 0, 433, - 0, 435, 436, 437, 438, 439, 0, 440, 441, 0, - 0, 442, 443, 444, 445, 446, 0, 447, 448, 449, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 468, 0, 469, 0, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, + 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 0, 413, 414, 415, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 511, 430, 431, 432, 0, 433, 434, + 0, 435, 0, 437, 438, 439, 440, 441, 0, 442, + 443, 444, 0, 0, 445, 446, 447, 448, 449, 0, + 450, 451, 452, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 471, 0, 472, 0, 474, + 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 834, 1408, 629, - 0, 0, 0, 996, 0, 0, 2618, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, - 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, - 0, 835, 0, 0, 0, 0, 836, 122, 123, 0, - 124, 125, 126, 837, 128, 129, 130, 838, 839, 840, - 841, 842, 0, 136, 137, 138, 139, 140, 141, 0, - 0, 142, 143, 843, 844, 146, 0, 147, 148, 149, - 150, 845, 0, 846, 0, 847, 154, 155, 156, 157, - 158, 848, 160, 161, 162, 0, 163, 164, 165, 166, - 167, 168, 0, 849, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 850, 851, 182, 1617, 183, 0, - 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, - 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, - 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 852, 211, 212, 213, 214, 215, 853, 1409, - 217, 0, 218, 219, 854, 221, 0, 222, 0, 223, - 855, 0, 856, 226, 227, 857, 858, 230, 0, 231, - 0, 859, 860, 234, 0, 235, 236, 237, 238, 239, - 240, 241, 861, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 862, 863, 257, - 258, 259, 260, 261, 864, 865, 0, 866, 0, 265, - 867, 868, 268, 869, 270, 271, 272, 273, 274, 275, - 0, 0, 276, 870, 278, 871, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 872, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 873, 874, 875, 310, 311, 312, - 876, 0, 314, 315, 877, 317, 0, 878, 319, 879, - 321, 322, 323, 0, 324, 325, 1410, 0, 326, 327, - 328, 0, 0, 329, 880, 881, 332, 882, 883, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 884, 885, 349, 350, 886, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 887, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 888, 385, 386, 387, - 889, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 890, 400, 401, 402, 403, 404, 405, - 891, 407, 408, 409, 892, 411, 412, 893, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 894, 428, 895, 430, 0, 431, 432, 0, - 433, 896, 435, 436, 437, 438, 439, 0, 897, 898, - 0, 0, 442, 443, 899, 445, 900, 1411, 447, 448, - 901, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 1251, 903, 0, 469, 904, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, - 480, 481, 482, 905, 906, 907, 908, 909, 910, 911, - 912, 913, 914, 915, 494, 495, 496, 497, 834, 1408, - 629, 0, 0, 0, 996, 1412, 1413, 0, 0, 0, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 508, 0, 0, 0, 0, 0, 0, 0, 0, 2618, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, + 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, + 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, + 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, + 140, 141, 0, 0, 142, 143, 144, 145, 146, 0, + 147, 148, 149, 150, 151, 0, 0, 0, 153, 154, + 155, 156, 157, 158, 0, 160, 161, 162, 0, 163, + 164, 165, 166, 167, 168, 0, 0, 170, 171, 172, + 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, + 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, + 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, + 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, + 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, + 222, 0, 223, 0, 0, 0, 226, 227, 509, 0, + 230, 0, 231, 0, 232, 233, 234, 235, 0, 236, + 237, 238, 239, 240, 241, 242, 0, 244, 245, 246, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 0, 257, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 266, 0, 0, 269, 0, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 0, 279, 0, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 510, + 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, 307, 308, 0, + 310, 311, 312, 313, 314, 0, 315, 316, 0, 318, + 0, 319, 320, 321, 322, 323, 324, 0, 325, 326, + 0, 0, 327, 328, 329, 0, 0, 330, 331, 332, + 0, 334, 0, 336, 337, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 0, 0, 0, 0, 347, 348, + 349, 0, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 364, 365, 366, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, 0, + 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 511, 430, 431, + 432, 0, 433, 434, 0, 435, 0, 437, 438, 439, + 440, 441, 0, 442, 443, 444, 0, 0, 445, 446, + 447, 448, 449, 0, 450, 451, 452, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 0, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 470, 471, + 0, 472, 0, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, + 497, 498, 499, 500, 837, 1411, 632, 0, 0, 0, + 999, 0, 0, 2621, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, + 117, 0, 118, 119, 120, 0, 0, 0, 838, 0, + 0, 0, 0, 839, 122, 123, 0, 124, 125, 126, + 840, 128, 129, 130, 841, 842, 843, 844, 845, 0, + 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, + 846, 847, 146, 0, 147, 148, 149, 150, 848, 0, + 849, 0, 850, 154, 155, 156, 157, 158, 851, 160, + 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, + 852, 170, 171, 172, 173, 174, 175, 176, 177, 178, + 179, 853, 854, 182, 1620, 183, 0, 184, 185, 186, + 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, + 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 855, + 211, 212, 213, 214, 215, 856, 1412, 217, 0, 218, + 219, 857, 221, 0, 222, 0, 223, 858, 0, 859, + 226, 227, 860, 861, 230, 0, 231, 0, 862, 863, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 864, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 865, 866, 258, 259, 260, + 261, 262, 867, 868, 0, 869, 0, 266, 870, 871, + 269, 872, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 873, 279, 874, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 875, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 876, 877, 878, 311, 312, 313, 879, 0, + 315, 316, 880, 318, 0, 881, 320, 882, 322, 323, + 324, 0, 325, 326, 1413, 0, 327, 328, 329, 0, + 0, 330, 331, 883, 884, 334, 885, 886, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 887, 888, 351, 352, 889, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 890, 367, 368, 369, 370, 0, 371, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 891, 387, 388, 389, 892, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 893, 402, 403, 404, 405, 406, 407, 894, + 409, 410, 411, 895, 413, 414, 896, 416, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, + 428, 897, 430, 898, 432, 0, 433, 434, 0, 435, + 899, 437, 438, 439, 440, 441, 0, 442, 900, 901, + 0, 0, 445, 446, 902, 448, 903, 1414, 450, 451, + 904, 453, 454, 455, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 1254, 906, 0, 472, 907, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, + 483, 484, 485, 908, 909, 910, 911, 912, 913, 914, + 915, 916, 917, 918, 497, 498, 499, 500, 837, 1411, + 632, 0, 0, 0, 999, 1415, 1416, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 835, 0, 0, 0, 0, 836, 122, 123, - 0, 124, 125, 126, 837, 128, 129, 130, 838, 839, - 840, 841, 842, 0, 136, 137, 138, 139, 140, 141, - 0, 0, 142, 143, 843, 844, 146, 0, 147, 148, - 149, 150, 845, 0, 846, 0, 847, 154, 155, 156, - 157, 158, 848, 160, 161, 162, 0, 163, 164, 165, - 166, 167, 168, 0, 849, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 850, 851, 182, 1619, 183, + 0, 0, 838, 0, 0, 0, 0, 839, 122, 123, + 0, 124, 125, 126, 840, 128, 129, 130, 841, 842, + 843, 844, 845, 0, 136, 137, 138, 139, 140, 141, + 0, 0, 142, 143, 846, 847, 146, 0, 147, 148, + 149, 150, 848, 0, 849, 0, 850, 154, 155, 156, + 157, 158, 851, 160, 161, 162, 0, 163, 164, 165, + 166, 167, 168, 0, 852, 170, 171, 172, 173, 174, + 175, 176, 177, 178, 179, 853, 854, 182, 1622, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 852, 211, 212, 213, 214, 215, 853, - 1409, 217, 0, 218, 219, 854, 221, 0, 222, 0, - 223, 855, 0, 856, 226, 227, 857, 858, 230, 0, - 231, 0, 859, 860, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 861, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 862, 863, - 257, 258, 259, 260, 261, 864, 865, 0, 866, 0, - 265, 867, 868, 268, 869, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 870, 278, 871, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 872, 289, 290, 291, + 207, 208, 209, 855, 211, 212, 213, 214, 215, 856, + 1412, 217, 0, 218, 219, 857, 221, 0, 222, 0, + 223, 858, 0, 859, 226, 227, 860, 861, 230, 0, + 231, 0, 862, 863, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 864, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 865, + 866, 258, 259, 260, 261, 262, 867, 868, 0, 869, + 0, 266, 870, 871, 269, 872, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 873, 279, 874, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 875, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 873, 874, 875, 310, 311, - 312, 876, 0, 314, 315, 877, 317, 0, 878, 319, - 879, 321, 322, 323, 0, 324, 325, 1410, 0, 326, - 327, 328, 0, 0, 329, 880, 881, 332, 882, 883, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 884, 885, 349, 350, - 886, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 887, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 888, 385, 386, - 387, 889, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 890, 400, 401, 402, 403, 404, - 405, 891, 407, 408, 409, 892, 411, 412, 893, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 894, 428, 895, 430, 0, 431, 432, - 0, 433, 896, 435, 436, 437, 438, 439, 0, 897, - 898, 0, 0, 442, 443, 899, 445, 900, 1411, 447, - 448, 901, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 1251, 903, 0, 469, 904, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 905, 906, 907, 908, 909, 910, - 911, 912, 913, 914, 915, 494, 495, 496, 497, 834, - 1408, 629, 0, 0, 0, 996, 1412, 1413, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, - 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 835, 0, 0, 0, 0, 836, 122, - 123, 0, 124, 125, 126, 837, 128, 129, 130, 838, - 839, 840, 841, 842, 0, 136, 137, 138, 139, 140, - 141, 0, 0, 142, 143, 843, 844, 146, 0, 147, - 148, 149, 150, 845, 0, 846, 0, 847, 154, 155, - 156, 157, 158, 848, 160, 161, 162, 0, 163, 164, - 165, 166, 167, 168, 0, 849, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 850, 851, 182, 0, - 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, - 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, - 206, 207, 208, 209, 852, 211, 212, 213, 214, 215, - 853, 1409, 217, 0, 218, 219, 854, 221, 0, 222, - 0, 223, 855, 0, 856, 226, 227, 857, 858, 230, - 0, 231, 0, 859, 860, 234, 0, 235, 236, 237, - 238, 239, 240, 241, 861, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 862, - 863, 257, 258, 259, 260, 261, 864, 865, 0, 866, - 0, 265, 867, 868, 268, 869, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 870, 278, 871, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 872, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 873, 874, 875, 310, - 311, 312, 876, 0, 314, 315, 877, 317, 0, 878, - 319, 879, 321, 322, 323, 0, 324, 325, 1410, 0, - 326, 327, 328, 0, 0, 329, 880, 881, 332, 882, - 883, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 884, 885, 349, - 350, 886, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 887, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 888, 385, - 386, 387, 889, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 0, 890, 400, 401, 402, 403, - 404, 405, 891, 407, 408, 409, 892, 411, 412, 893, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 894, 428, 895, 430, 0, 431, - 432, 0, 433, 896, 435, 436, 437, 438, 439, 0, - 897, 898, 0, 0, 442, 443, 899, 445, 900, 1411, - 447, 448, 901, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 1251, 903, 0, 469, 904, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 905, 906, 907, 908, 909, - 910, 911, 912, 913, 914, 915, 494, 495, 496, 497, - 0, 0, 1560, 0, 0, 1561, 0, 1412, 1413, 1562, - 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 0, 0, + 302, 303, 304, 305, 306, 307, 876, 877, 878, 311, + 312, 313, 879, 0, 315, 316, 880, 318, 0, 881, + 320, 882, 322, 323, 324, 0, 325, 326, 1413, 0, + 327, 328, 329, 0, 0, 330, 331, 883, 884, 334, + 885, 886, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 887, 888, + 351, 352, 889, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 890, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 891, + 387, 388, 389, 892, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 0, 893, 402, 403, 404, + 405, 406, 407, 894, 409, 410, 411, 895, 413, 414, + 896, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 897, 430, 898, 432, 0, + 433, 434, 0, 435, 899, 437, 438, 439, 440, 441, + 0, 442, 900, 901, 0, 0, 445, 446, 902, 448, + 903, 1414, 450, 451, 904, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 1254, 906, 0, 472, + 907, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 908, 909, 910, + 911, 912, 913, 914, 915, 916, 917, 918, 497, 498, + 499, 500, 837, 1411, 632, 0, 0, 0, 999, 1415, + 1416, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, + 118, 119, 120, 0, 0, 0, 838, 0, 0, 0, + 0, 839, 122, 123, 0, 124, 125, 126, 840, 128, + 129, 130, 841, 842, 843, 844, 845, 0, 136, 137, + 138, 139, 140, 141, 0, 0, 142, 143, 846, 847, + 146, 0, 147, 148, 149, 150, 848, 0, 849, 0, + 850, 154, 155, 156, 157, 158, 851, 160, 161, 162, + 0, 163, 164, 165, 166, 167, 168, 0, 852, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 853, + 854, 182, 0, 183, 0, 184, 185, 186, 187, 188, + 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, + 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, + 204, 205, 0, 206, 207, 208, 209, 855, 211, 212, + 213, 214, 215, 856, 1412, 217, 0, 218, 219, 857, + 221, 0, 222, 0, 223, 858, 0, 859, 226, 227, + 860, 861, 230, 0, 231, 0, 862, 863, 234, 235, + 0, 236, 237, 238, 239, 240, 241, 242, 864, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 865, 866, 258, 259, 260, 261, 262, + 867, 868, 0, 869, 0, 266, 870, 871, 269, 872, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 873, + 279, 874, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 875, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 876, 877, 878, 311, 312, 313, 879, 0, 315, 316, + 880, 318, 0, 881, 320, 882, 322, 323, 324, 0, + 325, 326, 1413, 0, 327, 328, 329, 0, 0, 330, + 331, 883, 884, 334, 885, 886, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 887, 888, 351, 352, 889, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 890, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 891, 387, 388, 389, 892, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 893, 402, 403, 404, 405, 406, 407, 894, 409, 410, + 411, 895, 413, 414, 896, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 897, + 430, 898, 432, 0, 433, 434, 0, 435, 899, 437, + 438, 439, 440, 441, 0, 442, 900, 901, 0, 0, + 445, 446, 902, 448, 903, 1414, 450, 451, 904, 453, + 454, 455, 456, 457, 0, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 1254, 906, 0, 472, 907, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, + 485, 908, 909, 910, 911, 912, 913, 914, 915, 916, + 917, 918, 497, 498, 499, 500, 0, 0, 1563, 0, + 0, 1564, 0, 1415, 1416, 1565, 1566, 1567, 1568, 1569, + 1570, 1571, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1572, 0, 0, 0, + 0, 2100, 0, 0, 0, 0, 1574, 0, 1563, 0, + 0, 1564, 0, 1575, 0, 1565, 1566, 1567, 1568, 1569, + 1570, 1571, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1572, 0, 1576, 0, + 0, 0, 0, 0, 0, 0, 1574, 0, 1563, 0, + 0, 1564, 0, 1575, 0, 1565, 1566, 1567, 1568, 1569, + 1570, 1571, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1572, 0, 1576, 0, + 0, 0, 2101, 0, 0, 0, 1574, 0, 1563, 0, + 0, 1564, 0, 1575, 0, 1565, 1566, 1567, 1568, 1569, + 1570, 1571, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1572, 0, 1576, 0, + 1857, 0, 0, 0, 0, 0, 1574, 0, 0, 0, + 0, 0, 1577, 1575, 0, 1563, 0, 0, 1564, 0, + 0, 0, 1565, 1566, 1567, 1568, 1569, 1570, 1571, 1578, + 0, 0, 0, 0, 1579, 0, 0, 0, 1576, 0, + 0, 0, 1893, 1572, 0, 0, 0, 1894, 0, 0, + 0, 0, 1577, 1574, 0, 0, 0, 1580, 1581, 0, + 1575, 0, 0, 0, 0, 0, 0, 0, 0, 1578, + 0, 0, 1582, 0, 1579, 0, 0, 0, 0, 0, + 0, 3390, 0, 0, 0, 1576, 0, 0, 0, 0, + 0, 0, 1577, 0, 0, 0, 0, 1580, 1581, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1583, 1578, + 0, 1584, 1582, 0, 1579, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1585, 0, 0, 1586, 0, + 0, 0, 1577, 0, 0, 0, 0, 1580, 1581, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1583, 1578, + 0, 1584, 1582, 0, 1579, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1585, 0, 0, 1586, 0, + 0, 0, 0, 0, 0, 0, 0, 1580, 1581, 1577, + 0, 0, 0, 0, 0, 0, 0, 0, 1583, 0, + 0, 1584, 1582, 0, 0, 0, 1578, 0, 0, 0, + 0, 1579, 0, 0, 0, 1585, 0, 0, 1586, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1569, 0, 0, 0, 0, 2097, 0, 0, 0, 0, - 1571, 0, 1560, 0, 0, 1561, 0, 1572, 0, 1562, - 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 0, 0, + 0, 0, 1587, 0, 1580, 1581, 0, 0, 1583, 0, + 0, 1584, 0, 0, 0, 0, 3391, 0, 0, 1582, + 0, 0, 0, 0, 0, 1585, 0, 0, 1586, 0, + 0, 0, 0, 0, 0, 0, 1563, 0, 0, 1564, + 0, 0, 1587, 1565, 1566, 1567, 1568, 1569, 1570, 1571, + 0, 0, 0, 0, 0, 1583, 0, 0, 1584, 0, + 0, 0, 0, 0, 1572, 2104, 0, 0, 1899, 0, + 0, 0, 1585, 0, 1574, 1586, 0, 0, 0, 0, + 0, 1575, 1587, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1864, 0, 0, + 0, 0, 0, 0, 0, 0, 1576, 1588, 0, 0, + 1589, 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, + 0, 0, 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1569, 0, 1573, 0, 0, 0, 0, 0, 0, 0, - 1571, 0, 1560, 0, 0, 1561, 0, 1572, 0, 1562, - 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1588, 0, 0, + 1589, 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1587, + 0, 0, 0, 1563, 0, 0, 1564, 0, 0, 0, + 1565, 1566, 1567, 1568, 1569, 1570, 1571, 1588, 0, 0, + 1589, 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, + 1577, 1572, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1574, 0, 0, 0, 0, 0, 1578, 1575, 0, + 0, 0, 1579, 0, 0, 0, 0, 1588, 0, 0, + 1589, 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, + 0, 0, 0, 1576, 0, 1580, 1581, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1569, 0, 1573, 0, 0, 0, 2098, 0, 0, 0, - 1571, 0, 1560, 0, 0, 1561, 0, 1572, 0, 1562, - 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 0, 0, + 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1588, 0, 0, 1589, 1590, 1591, + 0, 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1583, 1563, 0, 1584, + 1564, 0, 0, 0, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 0, 0, 1585, 0, 0, 1586, 0, 0, 0, + 0, 0, 0, 0, 0, 1572, 0, 0, 0, 1906, + 0, 0, 1904, 0, 0, 1574, 0, 1577, 0, 0, + 0, 0, 1575, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1578, 0, 0, 0, 0, 1579, + 0, 0, 0, 0, 0, 0, 0, 1576, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1569, 0, 1573, 0, 1854, 0, 0, 0, 0, 0, - 1571, 0, 1560, 0, 0, 1561, 1574, 1572, 0, 1562, - 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 0, 0, - 0, 0, 1575, 0, 0, 0, 0, 1576, 0, 0, - 1569, 0, 1573, 0, 0, 0, 1890, 0, 0, 0, - 1571, 1891, 0, 0, 0, 0, 1574, 1572, 0, 0, - 1577, 1578, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1575, 0, 0, 1579, 0, 1576, 0, 0, - 0, 0, 1573, 0, 0, 3385, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1574, 0, 0, 0, - 1577, 1578, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1580, 1575, 0, 1581, 1579, 0, 1576, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1582, 0, - 0, 1583, 0, 0, 0, 0, 1574, 0, 0, 0, - 1577, 1578, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1580, 1575, 0, 1581, 1579, 0, 1576, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1582, 0, - 0, 1583, 0, 0, 0, 0, 1574, 0, 0, 0, - 1577, 1578, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1580, 1575, 0, 1581, 1579, 0, 1576, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1582, 0, - 0, 1583, 0, 0, 0, 0, 0, 0, 0, 0, - 1577, 1578, 0, 0, 1584, 0, 0, 0, 0, 0, - 0, 1580, 0, 0, 1581, 1579, 0, 0, 3386, 0, + 0, 0, 1580, 1581, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1582, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1587, 0, 0, 0, 0, 0, 0, 1563, 0, 0, + 1564, 0, 0, 0, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 0, 0, 1583, 0, 0, 1584, 0, 0, 0, + 0, 0, 0, 0, 0, 1572, 0, 0, 0, 0, + 1585, 0, 0, 1586, 0, 1574, 0, 0, 0, 0, + 0, 1577, 1575, 0, 1563, 0, 0, 1564, 0, 0, + 0, 1565, 1566, 1567, 1568, 1569, 1570, 1571, 1578, 0, + 0, 0, 0, 1579, 0, 0, 0, 1576, 0, 0, + 0, 0, 1572, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1574, 0, 0, 0, 1580, 1581, 0, 1575, + 0, 0, 0, 0, 0, 1588, 0, 0, 1589, 1590, + 1591, 1582, 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, + 0, 0, 0, 0, 1576, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1587, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1583, 0, 0, + 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1996, 0, 0, 1585, 0, 0, 1586, 0, 0, + 0, 1577, 0, 0, 1563, 0, 0, 1564, 0, 0, + 0, 1565, 1566, 1567, 1568, 1569, 1570, 1571, 1578, 0, + 0, 0, 0, 1579, 0, 0, 0, 0, 0, 0, + 0, 0, 1572, 0, 0, 0, 2675, 0, 0, 0, + 0, 0, 1574, 0, 0, 0, 1580, 1581, 1577, 1575, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1582, 0, 0, 0, 1578, 0, 0, 0, 0, + 1579, 0, 1588, 0, 1576, 1589, 1590, 1591, 0, 1592, + 1593, 1594, 1595, 1596, 1597, 0, 0, 0, 0, 0, + 0, 1587, 0, 1580, 1581, 0, 0, 1583, 0, 0, + 1584, 0, 0, 0, 0, 0, 0, 0, 1582, 0, + 0, 0, 0, 0, 1585, 0, 0, 1586, 0, 0, + 0, 0, 1563, 0, 0, 1564, 0, 0, 0, 1565, + 1566, 1567, 1568, 1569, 1570, 1571, 0, 0, 0, 0, + 0, 0, 0, 0, 1583, 0, 0, 1584, 0, 0, + 1572, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1574, 1585, 0, 0, 1586, 0, 0, 1575, 1577, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1578, 0, 0, 0, 0, + 1579, 0, 1576, 0, 0, 0, 1588, 0, 0, 1589, + 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 0, + 0, 1587, 0, 1580, 1581, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1582, 0, - 0, 1583, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1584, 0, 0, 0, 0, 0, - 0, 1580, 0, 0, 1581, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2101, 0, 1582, 0, - 0, 1583, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1584, 0, 0, 0, 0, 0, - 0, 0, 1861, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1585, 0, - 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, - 1594, 0, 0, 0, 1584, 0, 0, 0, 0, 0, - 0, 0, 0, 1560, 0, 0, 1561, 0, 0, 0, - 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 1585, 0, - 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, - 1594, 1569, 0, 0, 1584, 1896, 0, 0, 0, 0, - 0, 1571, 0, 1560, 0, 0, 1561, 0, 1572, 0, - 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 1585, 0, - 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, - 1594, 1569, 0, 1573, 0, 0, 0, 0, 0, 0, - 0, 1571, 0, 1560, 0, 0, 1561, 0, 1572, 0, - 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 1585, 0, - 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, - 1594, 1569, 0, 1573, 0, 1903, 0, 0, 0, 0, - 0, 1571, 0, 1560, 0, 0, 1561, 0, 1572, 0, - 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 1585, 0, - 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, - 1594, 1569, 0, 1573, 0, 0, 0, 0, 0, 0, - 0, 1571, 0, 0, 0, 0, 0, 1574, 1572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1575, 0, 0, 0, 0, 1576, 0, - 0, 0, 0, 1573, 0, 0, 0, 0, 0, 0, - 0, 0, 1901, 0, 0, 0, 0, 1574, 0, 0, - 0, 1577, 1578, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1575, 0, 0, 1579, 0, 1576, 0, + 1563, 0, 0, 1564, 0, 0, 0, 1565, 1566, 1567, + 1568, 1569, 1570, 1571, 0, 0, 0, 0, 1587, 0, + 0, 0, 0, 0, 1583, 0, 0, 1584, 1572, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1574, 0, + 0, 1585, 0, 0, 1586, 1575, 1577, 0, 0, 0, + 0, 2664, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1578, 0, 0, 0, 0, 1579, 0, + 1576, 0, 0, 0, 0, 0, 1588, 0, 0, 1589, + 1590, 1591, 0, 1592, 1593, 1594, 1595, 1596, 1597, 0, + 0, 1580, 1581, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1574, 0, 0, - 0, 1577, 1578, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1580, 1575, 0, 1581, 1579, 0, 1576, 0, - 0, 0, 0, 0, 0, 0, 0, 1993, 0, 1582, - 0, 0, 1583, 0, 0, 0, 0, 1574, 0, 0, - 0, 1577, 1578, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1580, 1575, 0, 1581, 1579, 0, 1576, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1582, - 0, 0, 1583, 0, 0, 0, 0, 0, 0, 0, - 0, 1577, 1578, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1580, 0, 0, 1581, 1579, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1582, - 0, 0, 1583, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1584, 0, 0, 0, 0, - 0, 0, 1580, 0, 0, 1581, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1582, - 0, 0, 1583, 0, 0, 0, 0, 0, 0, 0, - 0, 1560, 0, 0, 1561, 1584, 0, 0, 1562, 1563, - 1564, 1565, 1566, 1567, 1568, 0, 0, 0, 0, 0, - 0, 0, 0, 1560, 0, 0, 1561, 0, 0, 1569, - 1562, 1563, 1564, 1565, 1566, 1567, 1568, 0, 0, 1571, - 0, 0, 0, 0, 0, 1584, 1572, 0, 0, 0, - 0, 1569, 0, 0, 0, 2672, 0, 0, 0, 0, - 0, 1571, 0, 0, 0, 0, 0, 0, 1572, 1585, - 0, 1573, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, - 1593, 1594, 0, 0, 0, 1584, 0, 0, 0, 0, - 0, 0, 0, 1573, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1588, 0, 0, 1589, 1590, 1591, 0, + 1592, 1593, 1594, 1595, 1596, 1597, 0, 0, 1587, 0, + 0, 0, 1583, 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1585, - 0, 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, - 1593, 1594, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1586, 0, 1577, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1585, - 0, 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, - 1593, 1594, 0, 0, 0, 1574, 0, 1560, 0, 0, - 1561, 0, 0, 0, 1562, 1563, 1564, 1565, 1566, 1567, - 1568, 1575, 0, 0, 0, 0, 1576, 1574, 0, 1585, - 0, 0, 1586, 1587, 1588, 1569, 1589, 1590, 1591, 1592, - 1593, 1594, 0, 1575, 0, 1571, 0, 0, 1576, 1577, - 1578, 0, 1572, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1579, 0, 0, 0, 0, 0, - 0, 1577, 1578, 0, 0, 0, 0, 1573, 0, 0, - 0, 0, 0, 0, 0, 0, 1579, 0, 0, 0, + 0, 1578, 0, 0, 0, 0, 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1580, 0, 0, 1581, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1582, 0, 0, - 1583, 0, 1580, 0, 0, 1581, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1582, - 0, 0, 1583, 0, 0, 0, 0, 0, 0, 0, - 1560, 0, 0, 1561, 0, 0, 0, 1562, 1563, 1564, - 1565, 1566, 1567, 1568, 0, 0, 0, 0, 0, 0, - 0, 1574, 1560, 0, 0, 1561, 0, 0, 1569, 1562, - 1563, 0, 0, 1566, 1567, 1568, 0, 1575, 1571, 0, - 0, 1560, 1576, 0, 1561, 1572, 0, 0, 1562, 1563, - 1569, 0, 1566, 1567, 1568, 0, 0, 0, 0, 0, - 1571, 0, 0, 1584, 0, 1577, 1578, 1572, 0, 0, - 1573, 0, 0, 0, 0, 0, 0, 0, 0, 1571, - 1579, 0, 0, 0, 0, 1584, 1572, 1560, 0, 0, - 1561, 0, 1573, 0, 1562, 1563, 2661, 0, 1566, 1567, - 1568, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1573, 0, 0, 0, 0, 1580, 0, 0, 1581, - 0, 0, 0, 0, 0, 1571, 0, 0, 0, 0, - 0, 0, 1572, 1582, 0, 0, 1583, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1580, + 1581, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1573, 0, 0, - 0, 0, 0, 0, 1574, 0, 0, 1585, 0, 0, - 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, 1594, - 1575, 0, 0, 0, 0, 1576, 1574, 0, 0, 1585, - 0, 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, - 1593, 1594, 1575, 0, 0, 1574, 0, 1576, 1577, 1578, + 0, 0, 0, 1588, 0, 0, 1589, 1590, 1591, 0, + 1592, 1593, 1594, 1595, 1596, 1597, 1587, 0, 0, 0, + 1583, 0, 0, 1584, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1585, 0, 0, + 1586, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1575, 0, 1579, 0, 0, 1576, 0, 0, 1584, - 1577, 1578, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1579, 0, 0, 0, 1577, - 1578, 1574, 0, 0, 0, 0, 0, 0, 0, 1580, - 0, 0, 1581, 0, 1579, 0, 0, 1575, 0, 0, - 0, 0, 1576, 0, 0, 0, 1582, 0, 0, 1583, - 0, 1580, 0, 0, 1581, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, -1968, -1968, 0, 1582, 0, - 1580, 0, 0, 1581, 0, 0, 0, 0, 0, 0, - 1579, 0, 0, 0, 0, 0, 0, 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1585, 0, 0, 1586, 1587, 1588, 0, - 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, 0, -1968, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1582, 0, 0, 0, 0, 0, 0, - 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1584, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1588, 0, 0, 1589, 1590, 1591, 0, 1592, 1593, + 1594, 1595, 1596, 1597, 1587, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1585, 0, 0, 1586, - 1587, 1588, 0, 1589, 1590, 1591, 1592, 2118, 1594, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1585, 0, - 0, 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, - 1594, 0, 0, 0, 0, 0, 0, 1585, 0, 0, - 1586, 1587, 1588, 0, 1589, 1590, 1591, 1592, 1593, 1594, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1186, 0, 2018, 0, 0, 0, 1588, + 0, 0, 1589, 1590, 1591, 0, 1592, 1593, 1594, 1595, + 2121, 1597, 110, 111, 112, 113, 114, 115, 116, 117, + 1187, 118, 119, 120, 1188, 1189, 1190, 838, 1191, 1192, + 1193, 1194, 839, 122, 123, 1195, 124, 125, 126, 840, + 128, 129, 130, 841, 842, 843, 844, 845, 1196, 136, + 137, 138, 139, 140, 141, 1197, 1198, 142, 143, 846, + 847, 146, 1199, 147, 148, 149, 150, 848, 1200, 849, + 1201, 850, 154, 155, 156, 157, 158, 851, 160, 161, + 162, 1202, 163, 164, 165, 166, 167, 168, 1203, 852, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 853, 854, 182, 1204, 183, 1205, 184, 185, 186, 187, + 188, 189, 1206, 190, 191, 192, 193, 194, 1207, 1208, + 195, 196, 197, 198, 199, 1209, 200, 201, 202, 1210, + 203, 204, 205, 1211, 206, 207, 208, 209, 855, 211, + 212, 213, 214, 215, 856, 1212, 217, 1213, 218, 219, + 857, 221, 1214, 222, 1215, 223, 858, 1216, 859, 226, + 227, 860, 861, 230, 1217, 231, 1218, 862, 863, 234, + 235, 1219, 236, 237, 238, 239, 240, 241, 242, 864, + 244, 245, 246, 247, 1220, 248, 249, 250, 251, 252, + 253, 254, 1221, 255, 865, 866, 258, 259, 260, 261, + 262, 867, 868, 1222, 869, 1223, 266, 870, 871, 269, + 872, 271, 272, 273, 274, 275, 276, 1224, 1225, 277, + 873, 279, 874, 1226, 281, 282, 283, 284, 285, 286, + 287, 288, 875, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + 307, 876, 877, 878, 311, 312, 313, 879, 1227, 315, + 316, 880, 318, 1228, 881, 320, 882, 322, 323, 324, + 1229, 325, 326, 1230, 1231, 327, 328, 329, 1232, 1233, + 330, 331, 883, 884, 334, 885, 886, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 1234, 1235, 1236, + 1237, 347, 348, 887, 888, 351, 352, 889, 354, 355, + 356, 1238, 357, 358, 359, 360, 361, 362, 1239, 363, + 364, 365, 890, 367, 368, 369, 370, 1240, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 1241, 384, 385, 891, 387, 388, 389, 892, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 1242, 893, 402, 403, 404, 405, 406, 407, 894, 409, + 410, 411, 895, 413, 414, 896, 416, 1243, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 897, 430, 898, 432, 1244, 433, 434, 1245, 435, 899, + 437, 438, 439, 440, 441, 1246, 442, 900, 901, 1247, + 1248, 445, 446, 902, 448, 903, 1249, 450, 451, 904, + 453, 454, 455, 456, 457, 1250, 1251, 458, 459, 460, + 1252, 461, 462, 463, 464, 1253, 465, 466, 467, 468, + 469, 1254, 906, 1255, 472, 907, 474, 475, 476, 477, + 478, 1256, 1257, 479, 1258, 1259, 480, 481, 482, 483, + 484, 485, 908, 909, 910, 911, 912, 913, 914, 915, + 916, 917, 918, 497, 498, 499, 500, 1186, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1183, 0, 2015, - 0, 0, 0, 1585, 0, 0, 1586, 1587, 1588, 0, - 1589, 1590, 1591, 1592, 1593, 1594, 110, 111, 112, 113, - 114, 115, 116, 117, 1184, 118, 119, 120, 1185, 1186, - 1187, 835, 1188, 1189, 1190, 1191, 836, 122, 123, 1192, - 124, 125, 126, 837, 128, 129, 130, 838, 839, 840, - 841, 842, 1193, 136, 137, 138, 139, 140, 141, 1194, - 1195, 142, 143, 843, 844, 146, 1196, 147, 148, 149, - 150, 845, 1197, 846, 1198, 847, 154, 155, 156, 157, - 158, 848, 160, 161, 162, 1199, 163, 164, 165, 166, - 167, 168, 1200, 849, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 850, 851, 182, 1201, 183, 1202, - 184, 185, 186, 187, 188, 189, 1203, 190, 191, 192, - 193, 194, 1204, 1205, 195, 196, 197, 198, 199, 1206, - 200, 201, 202, 1207, 203, 204, 205, 1208, 206, 207, - 208, 209, 852, 211, 212, 213, 214, 215, 853, 1209, - 217, 1210, 218, 219, 854, 221, 1211, 222, 1212, 223, - 855, 1213, 856, 226, 227, 857, 858, 230, 1214, 231, - 1215, 859, 860, 234, 1216, 235, 236, 237, 238, 239, - 240, 241, 861, 243, 244, 245, 246, 1217, 247, 248, - 249, 250, 251, 252, 253, 1218, 254, 862, 863, 257, - 258, 259, 260, 261, 864, 865, 1219, 866, 1220, 265, - 867, 868, 268, 869, 270, 271, 272, 273, 274, 275, - 1221, 1222, 276, 870, 278, 871, 1223, 280, 281, 282, - 283, 284, 285, 286, 287, 872, 289, 290, 291, 292, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 1187, 118, 119, 120, 1188, 1189, + 1190, 838, 1191, 1192, 1193, 1194, 839, 122, 123, 1195, + 124, 125, 126, 840, 128, 129, 130, 841, 842, 843, + 844, 845, 1196, 136, 137, 138, 139, 140, 141, 1197, + 1198, 142, 143, 846, 847, 146, 1199, 147, 148, 149, + 150, 848, 1200, 849, 1201, 850, 154, 155, 156, 157, + 158, 851, 160, 161, 162, 1202, 163, 164, 165, 166, + 167, 168, 1203, 852, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 853, 854, 182, 1204, 183, 1205, + 184, 185, 186, 187, 188, 189, 1206, 190, 191, 192, + 193, 194, 1207, 1208, 195, 196, 197, 198, 199, 1209, + 200, 201, 202, 1210, 203, 204, 205, 1211, 206, 207, + 208, 209, 855, 211, 212, 213, 214, 215, 856, 1212, + 217, 1213, 218, 219, 857, 221, 1214, 222, 1215, 223, + 858, 1216, 859, 226, 227, 860, 861, 230, 1217, 231, + 1218, 862, 863, 234, 235, 1219, 236, 237, 238, 239, + 240, 241, 242, 864, 244, 245, 246, 247, 1220, 248, + 249, 250, 251, 252, 253, 254, 1221, 255, 865, 866, + 258, 259, 260, 261, 262, 867, 868, 1222, 869, 1223, + 266, 870, 871, 269, 872, 271, 272, 273, 274, 275, + 276, 1224, 1225, 277, 873, 279, 874, 1226, 281, 282, + 283, 284, 285, 286, 287, 288, 875, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 873, 874, 875, 310, 311, 312, - 876, 1224, 314, 315, 877, 317, 1225, 878, 319, 879, - 321, 322, 323, 1226, 324, 325, 1227, 1228, 326, 327, - 328, 1229, 1230, 329, 880, 881, 332, 882, 883, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 1231, - 1232, 1233, 1234, 345, 346, 884, 885, 349, 350, 886, - 352, 353, 354, 1235, 355, 356, 357, 358, 359, 360, - 1236, 361, 362, 363, 887, 365, 366, 367, 368, 1237, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 1238, 382, 383, 888, 385, 386, 387, - 889, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 1239, 890, 400, 401, 402, 403, 404, 405, - 891, 407, 408, 409, 892, 411, 412, 893, 414, 1240, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 894, 428, 895, 430, 1241, 431, 432, 1242, - 433, 896, 435, 436, 437, 438, 439, 1243, 897, 898, - 1244, 1245, 442, 443, 899, 445, 900, 1246, 447, 448, - 901, 450, 451, 452, 453, 454, 1247, 1248, 455, 456, - 457, 1249, 458, 459, 460, 461, 1250, 462, 463, 464, - 465, 466, 1251, 903, 1252, 469, 904, 471, 472, 473, - 474, 475, 1253, 1254, 476, 1255, 1256, 477, 478, 479, - 480, 481, 482, 905, 906, 907, 908, 909, 910, 911, - 912, 913, 914, 915, 494, 495, 496, 497, 1183, 0, + 303, 304, 305, 306, 307, 876, 877, 878, 311, 312, + 313, 879, 1227, 315, 316, 880, 318, 1228, 881, 320, + 882, 322, 323, 324, 1229, 325, 326, 1230, 1231, 327, + 328, 329, 1232, 1233, 330, 331, 883, 884, 334, 885, + 886, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 1234, 1235, 1236, 1237, 347, 348, 887, 888, 351, + 352, 889, 354, 355, 356, 1238, 357, 358, 359, 360, + 361, 362, 1239, 363, 364, 365, 890, 367, 368, 369, + 370, 1240, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 1241, 384, 385, 891, 387, + 388, 389, 892, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 1242, 893, 402, 403, 404, 405, + 406, 407, 894, 409, 410, 411, 895, 413, 414, 896, + 416, 1243, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 897, 430, 898, 432, 1244, 433, + 434, 1245, 435, 899, 437, 438, 439, 440, 441, 1246, + 442, 900, 901, 1247, 1248, 445, 446, 902, 448, 903, + 1249, 450, 451, 904, 453, 454, 455, 456, 457, 1250, + 1251, 458, 459, 460, 1252, 461, 462, 463, 464, 1253, + 465, 466, 467, 468, 469, 1254, 906, 1255, 472, 907, + 474, 475, 476, 477, 478, 1256, 1257, 479, 1258, 1259, + 480, 481, 482, 483, 484, 485, 908, 909, 910, 911, + 912, 913, 914, 915, 916, 917, 918, 497, 498, 499, + 500, 1186, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, - 113, 114, 115, 116, 117, 1184, 118, 119, 120, 1185, - 1186, 1187, 835, 1188, 1189, 1190, 1191, 836, 122, 123, - 1192, 124, 125, 126, 837, 128, 129, 130, 838, 839, - 840, 841, 842, 1193, 136, 137, 138, 139, 140, 141, - 1194, 1195, 142, 143, 843, 844, 146, 1196, 147, 148, - 149, 150, 845, 1197, 846, 1198, 847, 154, 155, 156, - 157, 158, 848, 160, 161, 162, 1199, 163, 164, 165, - 166, 167, 168, 1200, 849, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 850, 851, 182, 1201, 183, - 1202, 184, 185, 186, 187, 188, 189, 1203, 190, 191, - 192, 193, 194, 1204, 1205, 195, 196, 197, 198, 199, - 1206, 200, 201, 202, 1207, 203, 204, 205, 1208, 206, - 207, 208, 209, 852, 211, 212, 213, 214, 215, 853, - 1209, 217, 1210, 218, 219, 854, 221, 1211, 222, 1212, - 223, 855, 1213, 856, 226, 227, 857, 858, 230, 1214, - 231, 1215, 859, 860, 234, 1216, 235, 236, 237, 238, - 239, 240, 241, 861, 243, 244, 245, 246, 1217, 247, - 248, 249, 250, 251, 252, 253, 1218, 254, 862, 863, - 257, 258, 259, 260, 261, 864, 865, 1219, 866, 1220, - 265, 867, 868, 268, 869, 270, 271, 272, 273, 274, - 275, 1221, 1222, 276, 870, 278, 871, 1223, 280, 281, - 282, 283, 284, 285, 286, 287, 872, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 873, 874, 875, 310, 311, - 312, 876, 1224, 314, 315, 877, 317, 1225, 878, 319, - 879, 321, 322, 323, 1226, 324, 325, 1227, 1228, 326, - 327, 328, 1229, 1230, 329, 880, 881, 332, 882, 883, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 1231, 1232, 1233, 1234, 345, 346, 884, 885, 349, 350, - 886, 352, 353, 354, 1235, 355, 356, 357, 358, 359, - 360, 1236, 361, 362, 363, 887, 365, 366, 367, 368, - 1237, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 1238, 382, 383, 888, 385, 386, - 387, 889, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 1239, 890, 400, 401, 402, 403, 404, - 405, 891, 407, 408, 409, 892, 411, 412, 893, 414, - 1240, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 894, 428, 895, 430, 1241, 431, 432, - 1242, 433, 896, 435, 436, 437, 438, 439, 1243, 897, - 898, 1244, 1245, 442, 443, 899, 445, 900, 1246, 447, - 448, 901, 450, 451, 452, 453, 454, 1247, 1248, 455, - 456, 457, 1249, 458, 459, 460, 461, 1250, 462, 463, - 464, 465, 466, 1251, 903, 1252, 469, 904, 471, 472, - 473, 474, 475, 1253, 1254, 476, 1255, 1256, 477, 478, - 479, 480, 481, 482, 905, 906, 907, 908, 909, 910, - 911, 912, 913, 914, 915, 494, 495, 496, 497, 1183, + 110, 111, 112, 113, 2166, 115, 116, 117, 1187, 118, + 119, 120, 1188, 1189, 1190, 838, 1191, 1192, 1193, 1194, + 839, 122, 123, 1195, 124, 125, 126, 840, 128, 129, + 130, 841, 842, 843, 844, 845, 1196, 136, 137, 138, + 139, 140, 141, 1197, 1198, 142, 143, 846, 847, 146, + 1199, 147, 148, 149, 150, 848, 1200, 849, 1201, 850, + 154, 155, 156, 157, 158, 851, 160, 161, 162, 1202, + 163, 164, 165, 166, 167, 168, 1203, 852, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 853, 854, + 182, 1204, 183, 1205, 184, 185, 186, 187, 188, 189, + 1206, 190, 191, 192, 193, 194, 1207, 1208, 195, 196, + 197, 2167, 199, 1209, 200, 201, 202, 1210, 203, 204, + 205, 1211, 206, 207, 208, 209, 855, 211, 212, 213, + 214, 215, 856, 1212, 217, 1213, 218, 219, 857, 221, + 1214, 222, 1215, 223, 858, 1216, 859, 226, 227, 860, + 861, 230, 1217, 231, 1218, 862, 863, 234, 235, 1219, + 236, 237, 238, 239, 240, 241, 242, 864, 244, 245, + 246, 247, 1220, 248, 249, 250, 251, 252, 253, 254, + 1221, 255, 865, 866, 258, 259, 260, 261, 262, 867, + 868, 1222, 869, 1223, 266, 870, 871, 269, 872, 271, + 272, 273, 274, 275, 276, 1224, 1225, 277, 873, 279, + 874, 1226, 281, 282, 283, 284, 285, 286, 287, 288, + 875, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 876, + 877, 878, 311, 312, 313, 879, 1227, 315, 316, 880, + 318, 1228, 881, 320, 882, 322, 323, 324, 1229, 325, + 326, 1230, 1231, 327, 328, 329, 1232, 1233, 330, 331, + 883, 884, 334, 885, 886, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 1234, 1235, 1236, 1237, 347, + 348, 887, 888, 351, 352, 889, 354, 355, 356, 1238, + 357, 358, 359, 360, 361, 362, 1239, 363, 364, 365, + 890, 367, 368, 369, 370, 1240, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 1241, + 384, 385, 891, 387, 388, 389, 892, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 1242, 893, + 402, 403, 404, 405, 406, 2168, 894, 409, 410, 411, + 895, 413, 414, 896, 416, 1243, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 897, 430, + 898, 432, 1244, 433, 434, 1245, 435, 899, 437, 438, + 439, 440, 441, 1246, 442, 900, 901, 1247, 1248, 445, + 446, 902, 448, 903, 1249, 450, 451, 904, 453, 454, + 455, 456, 457, 1250, 1251, 458, 459, 460, 1252, 461, + 462, 463, 464, 1253, 465, 466, 467, 468, 469, 1254, + 906, 1255, 472, 907, 474, 475, 476, 477, 478, 1256, + 1257, 479, 1258, 1259, 480, 481, 482, 483, 484, 485, + 908, 909, 910, 911, 912, 913, 914, 915, 916, 917, + 918, 497, 498, 499, 500, 837, 0, 632, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, + 116, 117, 0, 118, 119, 120, 0, 0, 0, 838, + 0, 0, 0, 0, 839, 122, 123, 0, 124, 125, + 126, 840, 128, 129, 130, 841, 842, 843, 844, 845, + 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, + 143, 846, 847, 146, 0, 147, 148, 149, 150, 848, + 0, 849, 0, 850, 154, 155, 156, 157, 158, 851, + 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, + 0, 852, 170, 171, 172, 173, 174, 175, 176, 177, + 178, 179, 853, 854, 182, 0, 183, 0, 184, 185, + 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, + 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, + 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, + 855, 211, 212, 213, 214, 215, 856, 1412, 217, 0, + 218, 219, 857, 221, 0, 222, 0, 223, 858, 0, + 859, 226, 227, 860, 861, 230, 0, 231, 0, 862, + 863, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 864, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 865, 866, 258, 259, + 260, 261, 262, 867, 868, 0, 869, 0, 266, 870, + 871, 269, 872, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 873, 279, 874, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 875, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 876, 877, 878, 311, 312, 313, 879, + 0, 315, 316, 880, 318, 0, 881, 320, 882, 322, + 323, 324, 0, 325, 326, 1413, 0, 327, 328, 329, + 0, 0, 330, 331, 883, 884, 334, 885, 886, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 887, 888, 351, 352, 889, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 890, 367, 368, 369, 370, 0, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 891, 387, 388, 389, + 892, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 0, 893, 402, 403, 404, 405, 406, 407, + 894, 409, 410, 411, 895, 413, 414, 896, 416, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 897, 430, 898, 432, 0, 433, 434, 0, + 435, 899, 437, 438, 439, 440, 441, 0, 442, 900, + 901, 0, 0, 445, 446, 902, 448, 903, 1414, 450, + 451, 904, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 1254, 906, 0, 472, 907, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, + 482, 483, 484, 485, 908, 909, 910, 911, 912, 913, + 914, 915, 916, 917, 918, 497, 498, 499, 500, 837, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, - 112, 113, 2163, 115, 116, 117, 1184, 118, 119, 120, - 1185, 1186, 1187, 835, 1188, 1189, 1190, 1191, 836, 122, - 123, 1192, 124, 125, 126, 837, 128, 129, 130, 838, - 839, 840, 841, 842, 1193, 136, 137, 138, 139, 140, - 141, 1194, 1195, 142, 143, 843, 844, 146, 1196, 147, - 148, 149, 150, 845, 1197, 846, 1198, 847, 154, 155, - 156, 157, 158, 848, 160, 161, 162, 1199, 163, 164, - 165, 166, 167, 168, 1200, 849, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 850, 851, 182, 1201, - 183, 1202, 184, 185, 186, 187, 188, 189, 1203, 190, - 191, 192, 193, 194, 1204, 1205, 195, 196, 197, 2164, - 199, 1206, 200, 201, 202, 1207, 203, 204, 205, 1208, - 206, 207, 208, 209, 852, 211, 212, 213, 214, 215, - 853, 1209, 217, 1210, 218, 219, 854, 221, 1211, 222, - 1212, 223, 855, 1213, 856, 226, 227, 857, 858, 230, - 1214, 231, 1215, 859, 860, 234, 1216, 235, 236, 237, - 238, 239, 240, 241, 861, 243, 244, 245, 246, 1217, - 247, 248, 249, 250, 251, 252, 253, 1218, 254, 862, - 863, 257, 258, 259, 260, 261, 864, 865, 1219, 866, - 1220, 265, 867, 868, 268, 869, 270, 271, 272, 273, - 274, 275, 1221, 1222, 276, 870, 278, 871, 1223, 280, - 281, 282, 283, 284, 285, 286, 287, 872, 289, 290, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 3, 4, 0, 838, 0, 0, 0, 0, 839, 122, + 123, 0, 124, 125, 126, 840, 128, 129, 130, 841, + 842, 843, 844, 845, 0, 136, 137, 138, 139, 140, + 141, 0, 0, 142, 143, 846, 847, 146, 0, 147, + 148, 149, 150, 848, 0, 849, 0, 850, 154, 155, + 156, 157, 158, 851, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 852, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 853, 854, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 855, 211, 212, 213, 214, 215, + 856, 0, 217, 0, 218, 219, 857, 221, 0, 222, + 0, 223, 858, 0, 859, 226, 227, 860, 861, 230, + 0, 231, 0, 862, 863, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 864, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 865, 866, 258, 259, 260, 261, 262, 867, 868, 0, + 869, 0, 266, 870, 871, 269, 872, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 873, 279, 874, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 875, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 873, 874, 875, 310, - 311, 312, 876, 1224, 314, 315, 877, 317, 1225, 878, - 319, 879, 321, 322, 323, 1226, 324, 325, 1227, 1228, - 326, 327, 328, 1229, 1230, 329, 880, 881, 332, 882, - 883, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 1231, 1232, 1233, 1234, 345, 346, 884, 885, 349, - 350, 886, 352, 353, 354, 1235, 355, 356, 357, 358, - 359, 360, 1236, 361, 362, 363, 887, 365, 366, 367, - 368, 1237, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 1238, 382, 383, 888, 385, - 386, 387, 889, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 1239, 890, 400, 401, 402, 403, - 404, 2165, 891, 407, 408, 409, 892, 411, 412, 893, - 414, 1240, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 894, 428, 895, 430, 1241, 431, - 432, 1242, 433, 896, 435, 436, 437, 438, 439, 1243, - 897, 898, 1244, 1245, 442, 443, 899, 445, 900, 1246, - 447, 448, 901, 450, 451, 452, 453, 454, 1247, 1248, - 455, 456, 457, 1249, 458, 459, 460, 461, 1250, 462, - 463, 464, 465, 466, 1251, 903, 1252, 469, 904, 471, - 472, 473, 474, 475, 1253, 1254, 476, 1255, 1256, 477, - 478, 479, 480, 481, 482, 905, 906, 907, 908, 909, - 910, 911, 912, 913, 914, 915, 494, 495, 496, 497, - 834, 0, 629, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, - 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, - 120, 0, 0, 0, 835, 0, 0, 0, 0, 836, - 122, 123, 0, 124, 125, 126, 837, 128, 129, 130, - 838, 839, 840, 841, 842, 0, 136, 137, 138, 139, - 140, 141, 0, 0, 142, 143, 843, 844, 146, 0, - 147, 148, 149, 150, 845, 0, 846, 0, 847, 154, - 155, 156, 157, 158, 848, 160, 161, 162, 0, 163, - 164, 165, 166, 167, 168, 0, 849, 170, 171, 172, - 173, 174, 175, 176, 177, 178, 179, 850, 851, 182, - 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, - 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 852, 211, 212, 213, 214, - 215, 853, 1409, 217, 0, 218, 219, 854, 221, 0, - 222, 0, 223, 855, 0, 856, 226, 227, 857, 858, - 230, 0, 231, 0, 859, 860, 234, 0, 235, 236, - 237, 238, 239, 240, 241, 861, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 862, 863, 257, 258, 259, 260, 261, 864, 865, 0, - 866, 0, 265, 867, 868, 268, 869, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 870, 278, 871, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 872, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 873, 874, 875, - 310, 311, 312, 876, 0, 314, 315, 877, 317, 0, - 878, 319, 879, 321, 322, 323, 0, 324, 325, 1410, - 0, 326, 327, 328, 0, 0, 329, 880, 881, 332, - 882, 883, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 884, 885, - 349, 350, 886, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 887, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 888, - 385, 386, 387, 889, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 890, 400, 401, 402, - 403, 404, 405, 891, 407, 408, 409, 892, 411, 412, - 893, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 894, 428, 895, 430, 0, - 431, 432, 0, 433, 896, 435, 436, 437, 438, 439, - 0, 897, 898, 0, 0, 442, 443, 899, 445, 900, - 1411, 447, 448, 901, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 1251, 903, 0, 469, 904, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 905, 906, 907, 908, - 909, 910, 911, 912, 913, 914, 915, 494, 495, 496, - 497, 834, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, - 119, 120, 3, 4, 0, 835, 0, 0, 0, 0, - 836, 122, 123, 0, 124, 125, 126, 837, 128, 129, - 130, 838, 839, 840, 841, 842, 0, 136, 137, 138, - 139, 140, 141, 0, 0, 142, 143, 843, 844, 146, - 0, 147, 148, 149, 150, 845, 0, 846, 0, 847, - 154, 155, 156, 157, 158, 848, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 849, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 850, 851, - 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, - 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 852, 211, 212, 213, - 214, 215, 853, 0, 217, 0, 218, 219, 854, 221, - 0, 222, 0, 223, 855, 0, 856, 226, 227, 857, - 858, 230, 0, 231, 0, 859, 860, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 861, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 862, 863, 257, 258, 259, 260, 261, 864, 865, - 0, 866, 0, 265, 867, 868, 268, 869, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 870, 278, 871, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 872, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 873, 874, - 875, 310, 311, 312, 876, 0, 314, 315, 877, 317, - 0, 878, 319, 879, 321, 322, 323, 0, 324, 325, - 0, 0, 326, 327, 328, 0, 0, 329, 880, 881, - 332, 882, 883, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 884, - 885, 349, 350, 886, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 887, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 888, 385, 386, 387, 889, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 890, 400, 401, - 402, 403, 404, 405, 891, 407, 408, 409, 892, 411, - 412, 893, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 894, 428, 895, 430, - 0, 431, 432, 0, 433, 896, 435, 436, 437, 438, - 439, 0, 897, 898, 0, 0, 442, 443, 899, 445, - 900, 0, 447, 448, 901, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 1251, 903, 0, 469, - 904, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 905, 906, 907, - 908, 909, 910, 911, 912, 913, 914, 915, 494, 495, - 496, 497, 109, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, - 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, - 129, 130, 131, 132, 133, 134, 135, 0, 136, 137, - 138, 139, 140, 141, 0, 599, 142, 143, 144, 145, - 146, 0, 147, 148, 149, 150, 600, 0, 601, 0, - 153, 154, 155, 156, 157, 158, 159, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 602, 0, 217, 0, 218, 219, 220, - 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, - 228, 229, 230, 0, 231, 0, 232, 233, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 255, 256, 257, 258, 259, 260, 261, 262, - 263, 0, 264, 0, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 277, 278, - 279, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, 313, 0, 314, 315, 316, - 317, 0, 604, 319, 320, 321, 322, 323, 0, 324, - 325, 0, 0, 326, 327, 328, 0, 0, 329, 330, - 331, 332, 333, 606, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 607, 348, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 364, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, - 411, 412, 413, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 609, - 430, 0, 431, 432, 0, 433, 434, 435, 436, 437, - 438, 439, 0, 440, 441, 0, 0, 442, 443, 610, - 445, 611, 0, 447, 448, 612, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 468, 0, - 469, 470, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 109, 0, 0, 0, 0, 0, 0, + 301, 302, 303, 304, 305, 306, 307, 876, 877, 878, + 311, 312, 313, 879, 0, 315, 316, 880, 318, 0, + 881, 320, 882, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 883, 884, + 334, 885, 886, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 887, + 888, 351, 352, 889, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 890, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 891, 387, 388, 389, 892, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 893, 402, 403, + 404, 405, 406, 407, 894, 409, 410, 411, 895, 413, + 414, 896, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 897, 430, 898, 432, + 0, 433, 434, 0, 435, 899, 437, 438, 439, 440, + 441, 0, 442, 900, 901, 0, 0, 445, 446, 902, + 448, 903, 0, 450, 451, 904, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 1254, 906, 0, + 472, 907, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 908, 909, + 910, 911, 912, 913, 914, 915, 916, 917, 918, 497, + 498, 499, 500, 109, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 121, 122, 123, 0, 124, 125, 126, 127, 128, 129, 130, 131, 132, 133, 134, 135, 0, 136, - 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, - 145, 146, 0, 147, 148, 149, 150, 151, 0, 152, + 137, 138, 139, 140, 141, 0, 602, 142, 143, 144, + 145, 146, 0, 147, 148, 149, 150, 603, 0, 604, 0, 153, 154, 155, 156, 157, 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, @@ -8902,377 +8910,331 @@ static const yytype_int16 yytable[] = 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, + 212, 213, 214, 215, 605, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, 224, 0, 225, 226, 227, 228, 229, 230, 0, 231, 0, 232, 233, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 0, 264, 0, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 277, - 278, 279, 0, 280, 281, 282, 283, 284, 285, 286, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 278, 279, 280, 0, 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 0, 314, 315, - 316, 317, 0, 318, 319, 320, 321, 322, 323, 0, - 324, 325, 0, 0, 326, 327, 328, 0, 0, 329, - 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 364, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 410, 411, 412, 413, 414, 0, 415, 416, 417, 418, + 307, 308, 309, 310, 311, 312, 313, 314, 0, 315, + 316, 317, 318, 0, 607, 320, 321, 322, 323, 324, + 0, 325, 326, 0, 0, 327, 328, 329, 0, 0, + 330, 331, 332, 333, 334, 335, 609, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 610, 350, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 429, 430, 0, 431, 432, 0, 433, 434, 435, 436, - 437, 438, 439, 0, 440, 441, 0, 0, 442, 443, - 444, 445, 446, 0, 447, 448, 449, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 468, - 0, 469, 470, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, + 429, 430, 612, 432, 0, 433, 434, 0, 435, 436, + 437, 438, 439, 440, 441, 0, 442, 443, 444, 0, + 0, 445, 446, 613, 448, 614, 0, 450, 451, 615, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 505, 0, 0, 0, 0, 0, + 494, 495, 496, 497, 498, 499, 500, 109, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, - 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, - 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, - 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, - 144, 145, 146, 1690, 147, 148, 149, 150, 151, 0, - 0, 1691, 153, 154, 155, 156, 157, 158, 0, 160, - 161, 162, 1692, 163, 164, 165, 166, 167, 168, 0, - 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, - 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, - 219, 220, 221, 0, 222, 1693, 223, 0, 0, 0, - 226, 227, 506, 0, 230, 0, 231, 0, 232, 233, - 234, 0, 235, 236, 237, 238, 239, 1694, 241, 0, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 0, 256, 257, 258, 259, 260, - 261, 262, 263, 0, 264, 0, 265, 0, 0, 268, - 0, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 0, 278, 0, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 507, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 0, 309, 310, 311, 312, 313, 0, 314, - 315, 0, 317, 0, 318, 319, 320, 321, 322, 323, - 0, 324, 325, 0, 0, 326, 327, 328, 0, 0, - 329, 330, 0, 332, 0, 334, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 347, 0, 349, 350, 351, 352, 353, 354, - 1695, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 364, 365, 366, 367, 368, 0, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 0, 385, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 0, 411, 412, 413, 414, 0, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 508, - 428, 429, 430, 0, 431, 432, 0, 433, 0, 435, - 436, 437, 438, 439, 0, 440, 441, 0, 0, 442, - 443, 444, 445, 446, 0, 447, 448, 449, 450, 451, - 452, 453, 454, 0, 1696, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 468, 0, 469, 0, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 505, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 0, 0, 0, 121, 122, 123, 0, + 124, 125, 126, 127, 128, 129, 130, 131, 132, 133, + 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, + 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, + 150, 151, 0, 152, 0, 153, 154, 155, 156, 157, + 158, 159, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 169, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, + 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, + 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, + 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, + 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, + 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, + 224, 0, 225, 226, 227, 228, 229, 230, 0, 231, + 0, 232, 233, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 256, 257, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 278, 279, 280, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, + 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, 314, 0, 315, 316, 317, 318, 0, 319, 320, + 321, 322, 323, 324, 0, 325, 326, 0, 0, 327, + 328, 329, 0, 0, 330, 331, 332, 333, 334, 335, + 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 366, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 431, 432, 0, 433, + 434, 0, 435, 436, 437, 438, 439, 440, 441, 0, + 442, 443, 444, 0, 0, 445, 446, 447, 448, 449, + 0, 450, 451, 452, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 471, 0, 472, 473, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, + 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, + 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, + 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, + 139, 140, 141, 0, 0, 142, 143, 144, 145, 146, + 1693, 147, 148, 149, 150, 151, 0, 0, 1694, 153, + 154, 155, 156, 157, 158, 0, 160, 161, 162, 1695, + 163, 164, 165, 166, 167, 168, 0, 0, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, + 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, + 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, + 0, 222, 1696, 223, 0, 0, 0, 226, 227, 509, + 0, 230, 0, 231, 0, 232, 233, 234, 235, 0, + 236, 237, 238, 239, 240, 1697, 242, 0, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 0, 257, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 0, 266, 0, 0, 269, 0, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 0, 279, + 0, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 0, 310, 311, 312, 313, 314, 0, 315, 316, 0, + 318, 0, 319, 320, 321, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 332, 0, 334, 0, 336, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 349, 0, 351, 352, 353, 354, 355, 356, 1698, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 511, 430, + 431, 432, 0, 433, 434, 0, 435, 0, 437, 438, + 439, 440, 441, 0, 442, 443, 444, 0, 0, 445, + 446, 447, 448, 449, 0, 450, 451, 452, 453, 454, + 455, 456, 457, 0, 1699, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 471, 0, 472, 0, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, - 143, 144, 145, 146, 1690, 147, 148, 149, 150, 151, + 143, 144, 145, 146, 1693, 147, 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, 0, - 160, 161, 162, 1692, 163, 164, 165, 166, 167, 168, + 160, 161, 162, 1695, 163, 164, 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, - 218, 219, 220, 221, 0, 222, 1693, 223, 0, 0, - 0, 226, 227, 506, 0, 230, 0, 231, 0, 232, - 233, 234, 0, 235, 236, 237, 238, 239, 240, 241, - 0, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 0, 256, 257, 258, 259, - 260, 261, 262, 263, 0, 264, 0, 265, 0, 0, - 268, 0, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 0, 278, 2237, 0, 280, 281, 282, 283, 284, - 285, 286, 287, 507, 289, 290, 291, 292, 293, 294, + 218, 219, 220, 221, 0, 222, 1696, 223, 0, 0, + 0, 226, 227, 509, 0, 230, 0, 231, 0, 232, + 233, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 0, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 0, 257, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 266, 0, + 0, 269, 0, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 0, 279, 2240, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 0, 309, 310, 311, 312, 313, 0, - 314, 315, 0, 317, 0, 318, 319, 320, 321, 322, - 323, 0, 324, 325, 0, 0, 326, 327, 328, 0, - 0, 329, 330, 0, 332, 0, 334, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 0, 0, 0, - 0, 345, 346, 347, 0, 349, 350, 351, 352, 353, - 354, 1695, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 364, 365, 366, 367, 368, 0, 369, 370, + 305, 306, 307, 308, 0, 310, 311, 312, 313, 314, + 0, 315, 316, 0, 318, 0, 319, 320, 321, 322, + 323, 324, 0, 325, 326, 0, 0, 327, 328, 329, + 0, 0, 330, 331, 332, 0, 334, 0, 336, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 349, 0, 351, 352, 353, + 354, 355, 356, 1698, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 0, 385, 386, 387, 388, 389, + 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 0, 0, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, 0, 411, 412, 413, 414, 0, 415, 416, + 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 508, 428, 429, 430, 0, 431, 432, 0, 433, 0, - 435, 436, 437, 438, 439, 0, 440, 441, 0, 0, - 442, 443, 444, 445, 446, 0, 447, 448, 449, 450, - 451, 452, 453, 454, 0, 1696, 455, 456, 457, 0, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 467, 468, 0, 469, 0, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 0, 477, 478, 479, 480, 481, + 427, 428, 511, 430, 431, 432, 0, 433, 434, 0, + 435, 0, 437, 438, 439, 440, 441, 0, 442, 443, + 444, 0, 0, 445, 446, 447, 448, 449, 0, 450, + 451, 452, 453, 454, 455, 456, 457, 0, 1699, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 471, 0, 472, 0, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 505, 0, 527, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 0, 118, 119, 120, 3, 4, 0, - 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, - 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, - 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, - 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, - 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, - 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, - 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, - 0, 0, 226, 227, 506, 0, 230, 0, 231, 0, - 232, 233, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 0, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 0, 256, 257, 258, - 259, 260, 261, 262, 263, 0, 264, 0, 265, 0, - 0, 268, 0, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 0, 278, 0, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 507, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 0, 309, 310, 311, 312, 313, - 0, 314, 315, 0, 317, 0, 318, 319, 320, 321, - 322, 323, 0, 324, 325, 0, 0, 326, 327, 328, - 0, 0, 329, 330, 0, 332, 0, 334, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 347, 0, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 364, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 0, 385, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 0, 411, 412, 413, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 508, 428, 429, 430, 0, 431, 432, 0, 433, - 0, 435, 436, 437, 438, 439, 0, 440, 441, 0, - 0, 442, 443, 444, 445, 446, 0, 447, 448, 449, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 468, 0, 469, 0, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 505, 0, 527, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 508, + 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 3, 4, 0, 0, 0, 0, 0, 0, 0, 122, + 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, + 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, + 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, + 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, + 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, + 0, 223, 0, 0, 0, 226, 227, 509, 0, 230, + 0, 231, 0, 232, 233, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 0, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 0, 257, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 266, 0, 0, 269, 0, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 0, 279, 0, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 0, 310, + 311, 312, 313, 314, 0, 315, 316, 0, 318, 0, + 319, 320, 321, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 0, + 334, 0, 336, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 349, + 0, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 366, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 0, 413, + 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 511, 430, 431, 432, + 0, 433, 434, 0, 435, 0, 437, 438, 439, 440, + 441, 0, 442, 443, 444, 0, 0, 445, 446, 447, + 448, 449, 0, 450, 451, 452, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 471, 0, + 472, 0, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, - 114, 115, 116, 117, 528, 118, 119, 120, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, - 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, - 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, - 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, - 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, - 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, - 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, - 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, - 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, - 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, - 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, - 0, 0, 0, 226, 227, 506, 0, 230, 0, 231, - 0, 232, 233, 234, 0, 235, 236, 237, 238, 239, - 240, 241, 0, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 0, 256, 257, - 258, 259, 260, 261, 262, 263, 0, 264, 0, 265, - 0, 0, 268, 0, 270, 271, 272, 273, 274, 275, - 0, 0, 276, 0, 278, 0, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 507, 289, 290, 291, 292, + 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, + 531, 118, 119, 120, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, + 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, + 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, + 145, 146, 0, 147, 148, 149, 150, 151, 0, 0, + 0, 153, 154, 155, 156, 157, 158, 0, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 0, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, + 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, + 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, + 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, + 220, 221, 0, 222, 0, 223, 0, 0, 0, 226, + 227, 509, 0, 230, 0, 231, 0, 232, 233, 234, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 0, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 0, 257, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 266, 0, 0, 269, + 0, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 0, 279, 0, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + 307, 308, 0, 310, 311, 312, 313, 314, 0, 315, + 316, 0, 318, 0, 319, 320, 321, 322, 323, 324, + 0, 325, 326, 0, 0, 327, 328, 329, 0, 0, + 330, 331, 332, 0, 334, 0, 336, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 349, 0, 351, 352, 353, 532, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 511, 430, 431, 432, 0, 433, 434, 0, 435, 0, + 437, 438, 439, 440, 441, 0, 442, 443, 444, 0, + 0, 445, 446, 447, 448, 449, 0, 450, 451, 452, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 471, 0, 472, 0, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, + 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 494, 495, 496, 497, 498, 499, 500, 508, 0, 530, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, + 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, + 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, + 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, + 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, + 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, + 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, + 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, + 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, + 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, + 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, + 0, 0, 0, 226, 227, 509, 0, 230, 0, 231, + 0, 232, 233, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 0, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 0, 257, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, + 266, 0, 0, 269, 0, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 0, 279, 0, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 0, 309, 310, 311, 312, - 313, 0, 314, 315, 0, 317, 0, 318, 319, 320, - 321, 322, 323, 0, 324, 325, 0, 0, 326, 327, - 328, 0, 0, 329, 330, 0, 332, 0, 334, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 347, 0, 349, 350, 351, - 529, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 364, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 0, 385, 386, 387, + 303, 304, 305, 306, 307, 308, 0, 310, 311, 312, + 313, 314, 0, 315, 316, 0, 318, 0, 319, 320, + 321, 322, 323, 324, 0, 325, 326, 0, 608, 327, + 328, 329, 0, 0, 330, 331, 332, 0, 334, 0, + 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 349, 0, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 366, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 0, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 0, 411, 412, 413, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 508, 428, 429, 430, 0, 431, 432, 0, - 433, 0, 435, 436, 437, 438, 439, 0, 440, 441, - 0, 0, 442, 443, 444, 445, 446, 0, 447, 448, - 449, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 467, 468, 0, 469, 0, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 0, 413, 414, 415, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 511, 430, 431, 432, 0, 433, + 434, 0, 435, 0, 437, 438, 439, 440, 441, 0, + 442, 443, 444, 0, 0, 445, 446, 447, 448, 449, + 0, 450, 451, 452, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 471, 0, 472, 0, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 505, 0, - 527, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, - 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, - 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, - 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, - 0, 0, 142, 143, 144, 145, 146, 0, 147, 148, - 149, 150, 151, 0, 0, 0, 153, 154, 155, 156, - 157, 158, 0, 160, 161, 162, 0, 163, 164, 165, - 166, 167, 168, 0, 0, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, - 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, - 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, - 223, 0, 0, 0, 226, 227, 506, 0, 230, 0, - 231, 0, 232, 233, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 0, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 0, 256, - 257, 258, 259, 260, 261, 262, 263, 0, 264, 0, - 265, 0, 0, 268, 0, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 0, 278, 0, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 507, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 0, 309, 310, 311, - 312, 313, 0, 314, 315, 0, 317, 0, 318, 319, - 320, 321, 322, 323, 0, 324, 325, 0, 605, 326, - 327, 328, 0, 0, 329, 330, 0, 332, 0, 334, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 347, 0, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 364, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 0, 385, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 0, 411, 412, 413, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 508, 428, 429, 430, 0, 431, 432, - 0, 433, 0, 435, 436, 437, 438, 439, 0, 440, - 441, 0, 0, 442, 443, 444, 445, 446, 0, 447, - 448, 449, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 468, 0, 469, 0, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 505, - 0, 527, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, - 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, - 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, - 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, - 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, - 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, - 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, - 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, - 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, - 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, - 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, - 0, 223, 0, 0, 0, 226, 227, 506, 0, 230, - 0, 231, 0, 232, 233, 234, 0, 235, 236, 237, - 238, 239, 709, 241, 0, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 0, - 256, 257, 258, 259, 260, 261, 262, 263, 0, 264, - 0, 265, 0, 0, 268, 0, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 0, 278, 0, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 507, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 0, 309, 310, - 311, 312, 313, 0, 314, 315, 0, 317, 0, 318, - 319, 320, 321, 322, 323, 0, 324, 325, 0, 605, - 326, 327, 328, 0, 0, 329, 330, 0, 332, 0, - 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 347, 0, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 364, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 0, 385, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 0, 0, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 0, 411, 412, 413, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 508, 428, 429, 430, 0, 431, - 432, 0, 433, 0, 435, 436, 437, 438, 439, 0, - 440, 441, 0, 0, 442, 443, 444, 445, 446, 0, - 447, 448, 449, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 467, 468, 0, 469, 0, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 505, 0, 527, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, - 111, 112, 113, 114, 115, 116, 117, 737, 118, 119, - 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, - 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, - 140, 141, 0, 0, 142, 143, 144, 145, 146, 0, - 147, 148, 149, 150, 151, 0, 0, 0, 153, 154, - 155, 156, 157, 158, 0, 160, 161, 162, 0, 163, - 164, 165, 166, 167, 168, 0, 0, 170, 171, 172, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, - 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, - 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, - 222, 0, 223, 0, 0, 0, 226, 227, 506, 0, - 230, 0, 231, 0, 232, 233, 234, 0, 235, 236, - 237, 238, 239, 240, 241, 0, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 0, 256, 257, 258, 259, 260, 261, 262, 263, 0, - 264, 0, 265, 0, 0, 268, 0, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 0, 278, 0, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 507, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 0, 309, - 310, 311, 312, 313, 0, 314, 315, 0, 317, 0, - 318, 319, 320, 321, 322, 323, 0, 324, 325, 0, - 0, 326, 327, 328, 0, 0, 329, 330, 0, 332, - 0, 334, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 347, 0, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 364, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 0, - 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 0, 411, 412, - 413, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 508, 428, 429, 430, 0, - 431, 432, 0, 433, 0, 435, 436, 437, 438, 439, - 0, 440, 441, 0, 0, 442, 443, 444, 445, 446, - 0, 447, 448, 449, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 468, 0, 469, 0, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 505, 1907, 0, 0, 0, 0, 1908, 0, 0, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, @@ -9288,87 +9250,136 @@ static const yytype_int16 yytable[] = 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, - 0, 222, 0, 223, 0, 0, 0, 226, 227, 506, - 0, 230, 0, 231, 0, 232, 233, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 0, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 0, 256, 257, 258, 259, 260, 261, 262, 263, - 0, 264, 0, 265, 0, 0, 268, 0, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 0, 278, 0, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 507, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 0, - 309, 310, 311, 312, 313, 0, 314, 315, 0, 317, - 0, 318, 319, 320, 321, 322, 323, 0, 324, 325, - 0, 0, 326, 327, 328, 0, 0, 329, 330, 0, - 332, 0, 334, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 347, - 0, 349, 350, 351, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 364, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 0, 385, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 0, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, 0, 411, - 412, 413, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 508, 428, 429, 430, - 0, 431, 432, 0, 433, 0, 435, 436, 437, 438, - 439, 0, 440, 441, 0, 0, 442, 443, 444, 445, - 446, 0, 447, 448, 449, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 467, 468, 0, 469, - 0, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 0, 222, 0, 223, 0, 0, 0, 226, 227, 509, + 0, 230, 0, 231, 0, 232, 233, 234, 235, 0, + 236, 237, 238, 239, 240, 712, 242, 0, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 0, 257, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 0, 266, 0, 0, 269, 0, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 0, 279, + 0, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 0, 310, 311, 312, 313, 314, 0, 315, 316, 0, + 318, 0, 319, 320, 321, 322, 323, 324, 0, 325, + 326, 0, 608, 327, 328, 329, 0, 0, 330, 331, + 332, 0, 334, 0, 336, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 349, 0, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 511, 430, + 431, 432, 0, 433, 434, 0, 435, 0, 437, 438, + 439, 440, 441, 0, 442, 443, 444, 0, 0, 445, + 446, 447, 448, 449, 0, 450, 451, 452, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 471, 0, 472, 0, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 505, 0, 527, 0, 0, 0, 0, 0, + 496, 497, 498, 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, - 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, - 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, - 146, 0, 147, 148, 149, 150, 151, 0, 0, 0, - 153, 154, 155, 156, 157, 158, 0, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 0, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, - 221, 0, 222, 0, 223, 0, 0, 0, 226, 227, - 506, 0, 1915, 0, 231, 0, 232, 233, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 0, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 0, 256, 257, 258, 259, 260, 261, 262, - 263, 0, 264, 0, 265, 0, 0, 268, 0, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 0, 278, - 0, 0, 280, 281, 1916, 283, 284, 285, 286, 287, - 507, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 0, 309, 310, 311, 312, 313, 0, 314, 315, 0, - 317, 0, 318, 319, 320, 321, 322, 323, 0, 324, - 325, 0, 0, 326, 327, 328, 0, 0, 329, 330, - 0, 332, 0, 334, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 347, 0, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 364, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 0, 385, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 0, - 411, 412, 413, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 508, 428, 429, - 430, 0, 431, 432, 1917, 433, 0, 435, 1918, 437, - 1919, 439, 0, 440, 441, 0, 0, 442, 443, 444, - 445, 446, 0, 447, 448, 449, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 1920, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 468, 0, - 469, 0, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 505, 0, 527, 0, 0, 0, 0, + 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, + 116, 117, 740, 118, 119, 120, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, + 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, + 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, + 143, 144, 145, 146, 0, 147, 148, 149, 150, 151, + 0, 0, 0, 153, 154, 155, 156, 157, 158, 0, + 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, + 0, 0, 170, 171, 172, 173, 174, 175, 176, 177, + 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, + 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, + 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, + 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, + 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, + 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, + 0, 226, 227, 509, 0, 230, 0, 231, 0, 232, + 233, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 0, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 0, 257, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 266, 0, + 0, 269, 0, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 0, 279, 0, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 0, 310, 311, 312, 313, 314, + 0, 315, 316, 0, 318, 0, 319, 320, 321, 322, + 323, 324, 0, 325, 326, 0, 0, 327, 328, 329, + 0, 0, 330, 331, 332, 0, 334, 0, 336, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 349, 0, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 366, 367, 368, 369, 370, 0, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 0, 413, 414, 415, 416, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 511, 430, 431, 432, 0, 433, 434, 0, + 435, 0, 437, 438, 439, 440, 441, 0, 442, 443, + 444, 0, 0, 445, 446, 447, 448, 449, 0, 450, + 451, 452, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 471, 0, 472, 0, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 508, + 1910, 0, 0, 0, 0, 1911, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, + 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, + 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, + 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, + 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, + 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, + 0, 223, 0, 0, 0, 226, 227, 509, 0, 230, + 0, 231, 0, 232, 233, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 0, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 0, 257, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 266, 0, 0, 269, 0, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 0, 279, 0, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 0, 310, + 311, 312, 313, 314, 0, 315, 316, 0, 318, 0, + 319, 320, 321, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 0, + 334, 0, 336, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 349, + 0, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 366, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 0, 413, + 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 511, 430, 431, 432, + 0, 433, 434, 0, 435, 0, 437, 438, 439, 440, + 441, 0, 442, 443, 444, 0, 0, 445, 446, 447, + 448, 449, 0, 450, 451, 452, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 471, 0, + 472, 0, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, @@ -9385,89 +9396,138 @@ static const yytype_int16 yytable[] = 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, 226, - 227, 506, 0, 230, 0, 231, 0, 232, 233, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 0, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 0, 256, 257, 258, 259, 260, 261, - 262, 263, 0, 264, 0, 265, 0, 0, 268, 0, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 0, - 278, 0, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 507, 289, 290, 291, 292, 293, 294, 295, 296, + 227, 509, 0, 1918, 0, 231, 0, 232, 233, 234, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 0, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 0, 257, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 266, 0, 0, 269, + 0, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 0, 279, 0, 0, 281, 282, 1919, 284, 285, 286, + 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 0, 309, 310, 311, 312, 313, 0, 314, 315, - 0, 317, 0, 318, 319, 320, 321, 322, 323, 0, - 324, 325, 0, 0, 326, 327, 328, 0, 0, 329, - 330, 0, 332, 0, 334, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 347, 0, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 364, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 0, 385, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 0, 411, 412, 413, 414, 0, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 508, 428, - 429, 430, 0, 431, 432, 0, 433, 0, 435, 436, - 437, 438, 439, 0, 440, 441, 0, 0, 442, 443, - 444, 445, 446, 0, 447, 448, 449, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 468, - 0, 469, 0, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, + 307, 308, 0, 310, 311, 312, 313, 314, 0, 315, + 316, 0, 318, 0, 319, 320, 321, 322, 323, 324, + 0, 325, 326, 0, 0, 327, 328, 329, 0, 0, + 330, 331, 332, 0, 334, 0, 336, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 349, 0, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 511, 430, 431, 432, 0, 433, 434, 1920, 435, 0, + 437, 1921, 439, 1922, 441, 0, 442, 443, 444, 0, + 0, 445, 446, 447, 448, 449, 0, 450, 451, 452, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 1923, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 471, 0, 472, 0, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 505, 0, 629, 0, 0, 0, + 494, 495, 496, 497, 498, 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, - 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, - 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, - 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, - 144, 145, 146, 0, 147, 148, 149, 150, 151, 0, - 0, 0, 153, 154, 155, 156, 157, 158, 0, 160, - 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, - 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, - 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, - 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, - 226, 227, 506, 0, 230, 0, 231, 0, 232, 233, - 234, 0, 235, 236, 237, 238, 239, 240, 241, 0, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 0, 256, 257, 258, 259, 260, - 261, 262, 263, 0, 264, 0, 265, 0, 0, 268, - 0, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 0, 278, 0, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 507, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 0, 309, 310, 311, 312, 313, 0, 314, - 315, 0, 317, 0, 318, 319, 320, 321, 322, 323, - 0, 324, 325, 0, 0, 326, 327, 328, 0, 0, - 329, 330, 0, 332, 0, 334, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 347, 0, 349, 350, 351, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 364, 365, 366, 367, 368, 0, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 0, 385, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 0, 411, 412, 413, 414, 0, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 508, - 428, 429, 430, 0, 431, 432, 0, 433, 0, 435, - 436, 437, 438, 439, 0, 440, 441, 0, 0, 442, - 443, 444, 445, 446, 0, 447, 448, 449, 450, 451, - 452, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 468, 0, 469, 0, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 505, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, + 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, + 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, + 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, + 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, + 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, + 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, + 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, + 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, + 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, + 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, + 0, 0, 0, 226, 227, 509, 0, 230, 0, 231, + 0, 232, 233, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 0, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 0, 257, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, + 266, 0, 0, 269, 0, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 0, 279, 0, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, + 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 308, 0, 310, 311, 312, + 313, 314, 0, 315, 316, 0, 318, 0, 319, 320, + 321, 322, 323, 324, 0, 325, 326, 0, 0, 327, + 328, 329, 0, 0, 330, 331, 332, 0, 334, 0, + 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 349, 0, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 366, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 0, 387, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 0, 413, 414, 415, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 511, 430, 431, 432, 0, 433, + 434, 0, 435, 0, 437, 438, 439, 440, 441, 0, + 442, 443, 444, 0, 0, 445, 446, 447, 448, 449, + 0, 450, 451, 452, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 471, 0, 472, 0, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, + 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 508, 0, 632, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, + 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, + 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, + 139, 140, 141, 0, 0, 142, 143, 144, 145, 146, + 0, 147, 148, 149, 150, 151, 0, 0, 0, 153, + 154, 155, 156, 157, 158, 0, 160, 161, 162, 0, + 163, 164, 165, 166, 167, 168, 0, 0, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, + 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, + 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, + 0, 222, 0, 223, 0, 0, 0, 226, 227, 509, + 0, 230, 0, 231, 0, 232, 233, 234, 235, 0, + 236, 237, 238, 239, 240, 241, 242, 0, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 0, 257, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 0, 266, 0, 0, 269, 0, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 0, 279, + 0, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 0, 310, 311, 312, 313, 314, 0, 315, 316, 0, + 318, 0, 319, 320, 321, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 332, 0, 334, 0, 336, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 349, 0, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 511, 430, + 431, 432, 0, 433, 434, 0, 435, 0, 437, 438, + 439, 440, 441, 0, 442, 443, 444, 0, 0, 445, + 446, 447, 448, 449, 0, 450, 451, 452, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 471, 0, 472, 0, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, - 116, 117, 636, 118, 119, 120, 0, 0, 0, 0, + 116, 117, 639, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, @@ -9481,279 +9541,184 @@ static const yytype_int16 yytable[] = 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, - 0, 226, 227, 506, 0, 637, 0, 231, 0, 232, - 233, 234, 0, 235, 236, 237, 238, 239, 240, 241, - 0, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 0, 256, 257, 258, 259, - 260, 261, 262, 263, 0, 264, 0, 265, 0, 0, - 268, 0, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 0, 278, 0, 0, 280, 281, 638, 283, 284, - 285, 286, 287, 507, 289, 290, 291, 292, 293, 294, + 0, 226, 227, 509, 0, 640, 0, 231, 0, 232, + 233, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 0, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 0, 257, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 266, 0, + 0, 269, 0, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 0, 279, 0, 0, 281, 282, 641, 284, + 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 0, 309, 310, 311, 312, 313, 0, - 314, 315, 0, 317, 0, 318, 319, 320, 321, 322, - 323, 0, 324, 325, 0, 0, 326, 327, 328, 0, - 0, 329, 330, 0, 332, 0, 334, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 0, 0, 0, - 0, 345, 346, 347, 0, 349, 350, 351, 352, 353, - 354, 0, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 364, 365, 366, 367, 368, 0, 369, 370, + 305, 306, 307, 308, 0, 310, 311, 312, 313, 314, + 0, 315, 316, 0, 318, 0, 319, 320, 321, 322, + 323, 324, 0, 325, 326, 0, 0, 327, 328, 329, + 0, 0, 330, 331, 332, 0, 334, 0, 336, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 349, 0, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 0, 385, 386, 387, 388, 389, + 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 0, 0, 400, 401, 402, 403, 639, 405, 406, 407, - 408, 409, 0, 411, 412, 413, 414, 0, 415, 416, + 400, 401, 0, 0, 402, 403, 404, 405, 642, 407, + 408, 409, 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 508, 428, 429, 430, 0, 431, 432, 0, 433, 0, - 435, 436, 437, 438, 439, 0, 640, 441, 0, 0, - 641, 443, 444, 445, 446, 0, 447, 448, 449, 450, - 451, 452, 453, 454, 0, 0, 455, 456, 457, 0, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 467, 468, 0, 469, 0, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 0, 477, 478, 479, 480, 481, + 427, 428, 511, 430, 431, 432, 0, 433, 434, 0, + 435, 0, 437, 438, 439, 440, 441, 0, 442, 643, + 444, 0, 0, 644, 446, 447, 448, 449, 0, 450, + 451, 452, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 471, 0, 472, 0, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 505, 0, 527, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, - 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, - 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, - 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, - 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, - 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, - 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, - 0, 0, 226, 227, 506, 0, 230, 0, 231, 0, - 232, 233, 234, 0, 235, 236, 237, 238, 239, 240, - 241, 0, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 0, 256, 257, 258, - 259, 260, 261, 262, 263, 0, 264, 0, 265, 0, - 0, 268, 0, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 0, 278, 0, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 507, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 0, 309, 310, 311, 312, 313, - 0, 314, 315, 0, 317, 0, 318, 319, 320, 321, - 322, 323, 0, 324, 325, 0, 0, 326, 327, 328, - 0, 0, 329, 330, 0, 332, 0, 334, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 347, 0, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 364, 365, 366, 673, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 0, 385, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 0, 411, 412, 413, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 508, 428, 429, 430, 0, 431, 432, 0, 433, - 0, 435, 436, 437, 438, 439, 0, 440, 441, 0, - 0, 442, 443, 444, 445, 446, 0, 447, 448, 449, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 468, 0, 469, 0, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 505, 0, 527, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 508, + 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, + 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, + 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, + 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, + 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, + 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, + 0, 223, 0, 0, 0, 226, 227, 509, 0, 230, + 0, 231, 0, 232, 233, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 0, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 0, 257, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 266, 0, 0, 269, 0, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 0, 279, 0, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 0, 310, + 311, 312, 313, 314, 0, 315, 316, 0, 318, 0, + 319, 320, 321, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 0, + 334, 0, 336, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 349, + 0, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 366, 367, + 368, 676, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 0, 413, + 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 511, 430, 431, 432, + 0, 433, 434, 0, 435, 0, 437, 438, 439, 440, + 441, 0, 442, 443, 444, 0, 0, 445, 446, 447, + 448, 449, 0, 450, 451, 452, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 471, 0, + 472, 0, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, - 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, - 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, - 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, - 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, - 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, - 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, - 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, + 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, + 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, + 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, + 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, + 145, 146, 0, 147, 148, 149, 150, 151, 0, 0, + 0, 153, 154, 155, 156, 157, 158, 0, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 0, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, + 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, + 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, + 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, + 220, 221, 0, 222, 0, 223, 0, 0, 0, 226, + 227, 509, 0, 230, 0, 231, 0, 232, 233, 234, + 235, 0, 236, 237, 238, 239, 240, 707, 242, 0, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 0, 257, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 266, 0, 0, 269, + 0, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 0, 279, 0, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + 307, 308, 0, 310, 311, 312, 313, 314, 0, 315, + 316, 0, 318, 0, 319, 320, 321, 322, 323, 324, + 0, 325, 326, 0, 0, 327, 328, 329, 0, 0, + 330, 331, 332, 0, 334, 0, 336, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 349, 0, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 511, 430, 431, 432, 0, 433, 434, 0, 435, 0, + 437, 438, 439, 440, 441, 0, 442, 443, 444, 0, + 0, 445, 446, 447, 448, 449, 0, 450, 451, 452, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 471, 0, 472, 0, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, + 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 494, 495, 496, 497, 498, 499, 500, 508, 0, 530, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, + 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, + 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, + 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, + 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, + 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, - 0, 0, 0, 226, 227, 506, 0, 230, 0, 231, - 0, 232, 233, 234, 0, 235, 236, 237, 238, 239, - 704, 241, 0, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 0, 256, 257, - 258, 259, 260, 261, 262, 263, 0, 264, 0, 265, - 0, 0, 268, 0, 270, 271, 272, 273, 274, 275, - 0, 0, 276, 0, 278, 0, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 507, 289, 290, 291, 292, + 0, 0, 0, 226, 227, 509, 0, 230, 0, 231, + 0, 232, 233, 234, 235, 0, 236, 237, 238, 239, + 240, 710, 242, 0, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 0, 257, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, + 266, 0, 0, 269, 0, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 0, 279, 0, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 0, 309, 310, 311, 312, - 313, 0, 314, 315, 0, 317, 0, 318, 319, 320, - 321, 322, 323, 0, 324, 325, 0, 0, 326, 327, - 328, 0, 0, 329, 330, 0, 332, 0, 334, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 347, 0, 349, 350, 351, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 364, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 0, 385, 386, 387, + 303, 304, 305, 306, 307, 308, 0, 310, 311, 312, + 313, 314, 0, 315, 316, 0, 318, 0, 319, 320, + 321, 322, 323, 324, 0, 325, 326, 0, 0, 327, + 328, 329, 0, 0, 330, 331, 332, 0, 334, 0, + 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 349, 0, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 366, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 0, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 0, 411, 412, 413, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 508, 428, 429, 430, 0, 431, 432, 0, - 433, 0, 435, 436, 437, 438, 439, 0, 440, 441, - 0, 0, 442, 443, 444, 445, 446, 0, 447, 448, - 449, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 467, 468, 0, 469, 0, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 0, 413, 414, 415, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 511, 430, 431, 432, 0, 433, + 434, 0, 435, 0, 437, 438, 439, 440, 441, 0, + 442, 443, 444, 0, 0, 445, 446, 447, 448, 449, + 0, 450, 451, 452, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 471, 0, 472, 0, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 505, 0, - 527, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, - 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, - 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, - 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, - 0, 0, 142, 143, 144, 145, 146, 0, 147, 148, - 149, 150, 151, 0, 0, 0, 153, 154, 155, 156, - 157, 158, 0, 160, 161, 162, 0, 163, 164, 165, - 166, 167, 168, 0, 0, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 0, 183, - 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 197, 198, 199, - 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, - 223, 0, 0, 0, 226, 227, 506, 0, 230, 0, - 231, 0, 232, 233, 234, 0, 235, 236, 237, 238, - 239, 707, 241, 0, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 0, 256, - 257, 258, 259, 260, 261, 262, 263, 0, 264, 0, - 265, 0, 0, 268, 0, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 0, 278, 0, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 507, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 0, 309, 310, 311, - 312, 313, 0, 314, 315, 0, 317, 0, 318, 319, - 320, 321, 322, 323, 0, 324, 325, 0, 0, 326, - 327, 328, 0, 0, 329, 330, 0, 332, 0, 334, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 347, 0, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 364, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 0, 385, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 0, 411, 412, 413, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 508, 428, 429, 430, 0, 431, 432, - 0, 433, 0, 435, 436, 437, 438, 439, 0, 440, - 441, 0, 0, 442, 443, 444, 445, 446, 0, 447, - 448, 449, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 468, 0, 469, 0, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 505, - 0, 527, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, - 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, - 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, - 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, - 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, - 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, - 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, - 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, - 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, - 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, - 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, - 0, 223, 0, 0, 0, 226, 227, 506, 0, 230, - 0, 231, 0, 232, 233, 234, 0, 235, 236, 237, - 238, 239, 711, 241, 0, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 0, - 256, 257, 258, 259, 260, 261, 262, 263, 0, 264, - 0, 265, 0, 0, 268, 0, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 0, 278, 0, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 507, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 0, 309, 310, - 311, 312, 313, 0, 314, 315, 0, 317, 0, 318, - 319, 320, 321, 322, 323, 0, 324, 325, 0, 0, - 326, 327, 328, 0, 0, 329, 330, 0, 332, 0, - 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 347, 0, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 364, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 0, 385, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 0, 0, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 0, 411, 412, 413, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 508, 428, 429, 430, 0, 431, - 432, 0, 433, 0, 435, 436, 437, 438, 439, 0, - 440, 441, 0, 0, 442, 443, 444, 445, 446, 0, - 447, 448, 449, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 467, 468, 0, 469, 0, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 505, 0, 527, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, - 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, - 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, - 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, - 140, 141, 0, 0, 142, 143, 144, 145, 146, 0, - 147, 148, 149, 150, 151, 0, 0, 0, 153, 154, - 155, 156, 157, 158, 0, 160, 161, 162, 0, 163, - 164, 165, 166, 167, 168, 0, 0, 170, 171, 172, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, - 190, 191, 192, 193, 194, 0, 0, 195, 196, 197, - 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, - 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, - 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, - 222, 0, 223, 0, 0, 0, 226, 227, 506, 0, - 230, 0, 231, 0, 232, 233, 234, 0, 235, 236, - 237, 238, 239, 726, 241, 0, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 0, 256, 257, 258, 259, 260, 261, 262, 263, 0, - 264, 0, 265, 0, 0, 268, 0, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 0, 278, 0, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 507, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 0, 309, - 310, 311, 312, 313, 0, 314, 315, 0, 317, 0, - 318, 319, 320, 321, 322, 323, 0, 324, 325, 0, - 0, 326, 327, 328, 0, 0, 329, 330, 0, 332, - 0, 334, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 347, 0, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 364, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 0, - 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 0, 411, 412, - 413, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 508, 428, 429, 430, 0, - 431, 432, 0, 433, 0, 435, 436, 437, 438, 439, - 0, 440, 441, 0, 0, 442, 443, 444, 445, 446, - 0, 447, 448, 449, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 468, 0, 469, 0, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 505, 0, 527, 0, 0, 0, 0, 0, 0, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, @@ -9769,91 +9734,140 @@ static const yytype_int16 yytable[] = 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, - 0, 222, 0, 223, 0, 0, 0, 226, 227, 506, - 0, 230, 0, 231, 0, 232, 233, 234, 0, 235, - 236, 237, 238, 239, 826, 241, 0, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 0, 256, 257, 258, 259, 260, 261, 262, 263, - 0, 264, 0, 265, 0, 0, 268, 0, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 0, 278, 0, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 507, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 0, - 309, 310, 311, 312, 313, 0, 314, 315, 0, 317, - 0, 318, 319, 320, 321, 322, 323, 0, 324, 325, - 0, 0, 326, 327, 328, 0, 0, 329, 330, 0, - 332, 0, 334, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 347, - 0, 349, 350, 351, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 364, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 0, 385, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 0, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, 0, 411, - 412, 413, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 508, 428, 429, 430, - 0, 431, 432, 0, 433, 0, 435, 436, 437, 438, - 439, 0, 440, 441, 0, 0, 442, 443, 444, 445, - 446, 0, 447, 448, 449, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 467, 468, 0, 469, - 0, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 0, 222, 0, 223, 0, 0, 0, 226, 227, 509, + 0, 230, 0, 231, 0, 232, 233, 234, 235, 0, + 236, 237, 238, 239, 240, 714, 242, 0, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 0, 257, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 0, 266, 0, 0, 269, 0, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 0, 279, + 0, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 0, 310, 311, 312, 313, 314, 0, 315, 316, 0, + 318, 0, 319, 320, 321, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 332, 0, 334, 0, 336, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 349, 0, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 511, 430, + 431, 432, 0, 433, 434, 0, 435, 0, 437, 438, + 439, 440, 441, 0, 442, 443, 444, 0, 0, 445, + 446, 447, 448, 449, 0, 450, 451, 452, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 471, 0, 472, 0, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 505, 0, 527, 0, 0, 0, 0, 0, + 496, 497, 498, 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, - 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, - 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, - 146, 0, 147, 148, 149, 150, 151, 0, 0, 0, - 153, 154, 155, 156, 157, 158, 0, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 0, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, - 221, 0, 222, 0, 223, 0, 0, 0, 226, 227, - 506, 0, 230, 0, 231, 0, 232, 233, 234, 0, - 235, 236, 237, 238, 239, 829, 241, 0, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 0, 256, 257, 258, 259, 260, 261, 262, - 263, 0, 264, 0, 265, 0, 0, 268, 0, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 0, 278, - 0, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 507, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 0, 309, 310, 311, 312, 313, 0, 314, 315, 0, - 317, 0, 318, 319, 320, 321, 322, 323, 0, 324, - 325, 0, 0, 326, 327, 328, 0, 0, 329, 330, - 0, 332, 0, 334, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 347, 0, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 364, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 0, 385, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, 0, - 411, 412, 413, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 508, 428, 429, - 430, 0, 431, 432, 0, 433, 0, 435, 436, 437, - 438, 439, 0, 440, 441, 0, 0, 442, 443, 444, - 445, 446, 0, 447, 448, 449, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 468, 0, - 469, 0, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 505, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, + 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, + 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, + 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, + 143, 144, 145, 146, 0, 147, 148, 149, 150, 151, + 0, 0, 0, 153, 154, 155, 156, 157, 158, 0, + 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, + 0, 0, 170, 171, 172, 173, 174, 175, 176, 177, + 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, + 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, + 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, + 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, + 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, + 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, + 0, 226, 227, 509, 0, 230, 0, 231, 0, 232, + 233, 234, 235, 0, 236, 237, 238, 239, 240, 729, + 242, 0, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 0, 257, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 266, 0, + 0, 269, 0, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 0, 279, 0, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 0, 310, 311, 312, 313, 314, + 0, 315, 316, 0, 318, 0, 319, 320, 321, 322, + 323, 324, 0, 325, 326, 0, 0, 327, 328, 329, + 0, 0, 330, 331, 332, 0, 334, 0, 336, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 349, 0, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 366, 367, 368, 369, 370, 0, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 0, 413, 414, 415, 416, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 511, 430, 431, 432, 0, 433, 434, 0, + 435, 0, 437, 438, 439, 440, 441, 0, 442, 443, + 444, 0, 0, 445, 446, 447, 448, 449, 0, 450, + 451, 452, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 471, 0, 472, 0, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 508, + 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, + 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, + 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, + 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, + 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, + 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, + 0, 223, 0, 0, 0, 226, 227, 509, 0, 230, + 0, 231, 0, 232, 233, 234, 235, 0, 236, 237, + 238, 239, 240, 829, 242, 0, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 0, 257, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 266, 0, 0, 269, 0, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 0, 279, 0, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 0, 310, + 311, 312, 313, 314, 0, 315, 316, 0, 318, 0, + 319, 320, 321, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 0, + 334, 0, 336, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 349, + 0, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 366, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 0, 413, + 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 511, 430, 431, 432, + 0, 433, 434, 0, 435, 0, 437, 438, 439, 440, + 441, 0, 442, 443, 444, 0, 0, 445, 446, 447, + 448, 449, 0, 450, 451, 452, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 471, 0, + 472, 0, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, - 950, 0, 0, 122, 123, 0, 124, 125, 126, 0, + 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, 151, 0, 0, @@ -9866,89 +9880,138 @@ static const yytype_int16 yytable[] = 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, 226, - 227, 506, 0, 230, 0, 231, 0, 232, 233, 234, - 0, 235, 236, 237, 238, 239, 240, 241, 0, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 0, 256, 257, 258, 259, 260, 261, - 262, 263, 0, 264, 0, 265, 0, 0, 268, 0, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 0, - 278, 0, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 507, 289, 290, 291, 292, 293, 294, 295, 296, + 227, 509, 0, 230, 0, 231, 0, 232, 233, 234, + 235, 0, 236, 237, 238, 239, 240, 832, 242, 0, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 0, 257, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 266, 0, 0, 269, + 0, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 0, 279, 0, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 0, 309, 310, 311, 312, 313, 0, 314, 315, - 0, 317, 0, 318, 319, 320, 321, 322, 323, 0, - 324, 325, 0, 0, 326, 327, 328, 0, 0, 329, - 330, 0, 332, 0, 334, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 347, 0, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 364, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 0, 385, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 0, 411, 412, 413, 414, 0, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 508, 428, - 429, 430, 0, 431, 432, 0, 433, 0, 435, 436, - 437, 438, 439, 0, 440, 441, 0, 0, 442, 443, - 444, 445, 446, 0, 447, 448, 449, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 468, - 0, 469, 0, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, + 307, 308, 0, 310, 311, 312, 313, 314, 0, 315, + 316, 0, 318, 0, 319, 320, 321, 322, 323, 324, + 0, 325, 326, 0, 0, 327, 328, 329, 0, 0, + 330, 331, 332, 0, 334, 0, 336, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 349, 0, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 511, 430, 431, 432, 0, 433, 434, 0, 435, 0, + 437, 438, 439, 440, 441, 0, 442, 443, 444, 0, + 0, 445, 446, 447, 448, 449, 0, 450, 451, 452, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 471, 0, 472, 0, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 505, 0, 0, 0, 0, 0, + 494, 495, 496, 497, 498, 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, - 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 0, 973, 0, 0, 122, 123, 0, 124, 125, 126, - 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, - 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, - 144, 145, 146, 0, 147, 148, 149, 150, 151, 0, - 0, 0, 153, 154, 155, 156, 157, 158, 0, 160, - 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, - 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, - 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, - 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, - 226, 227, 506, 0, 230, 0, 231, 0, 232, 233, - 234, 0, 235, 236, 237, 238, 239, 240, 241, 0, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 0, 256, 257, 258, 259, 260, - 261, 262, 263, 0, 264, 0, 265, 0, 0, 268, - 0, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 0, 278, 0, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 507, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 0, 309, 310, 311, 312, 313, 0, 314, - 315, 0, 317, 0, 318, 319, 320, 321, 322, 323, - 0, 324, 325, 0, 0, 326, 327, 328, 0, 0, - 329, 330, 0, 332, 0, 334, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 347, 0, 349, 350, 351, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 364, 365, 366, 367, 368, 0, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 0, 385, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 0, 411, 412, 413, 414, 0, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 508, - 428, 429, 430, 0, 431, 432, 0, 433, 0, 435, - 436, 437, 438, 439, 0, 440, 441, 0, 0, 442, - 443, 444, 445, 446, 0, 447, 448, 449, 450, 451, - 452, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 468, 0, 469, 0, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 505, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 0, 953, 0, 0, 122, 123, 0, + 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, + 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, + 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, + 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, + 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, + 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, + 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, + 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, + 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, + 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, + 0, 0, 0, 226, 227, 509, 0, 230, 0, 231, + 0, 232, 233, 234, 235, 0, 236, 237, 238, 239, + 240, 241, 242, 0, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 0, 257, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, + 266, 0, 0, 269, 0, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 0, 279, 0, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, + 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 308, 0, 310, 311, 312, + 313, 314, 0, 315, 316, 0, 318, 0, 319, 320, + 321, 322, 323, 324, 0, 325, 326, 0, 0, 327, + 328, 329, 0, 0, 330, 331, 332, 0, 334, 0, + 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 349, 0, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 366, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 0, 387, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 0, 413, 414, 415, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 511, 430, 431, 432, 0, 433, + 434, 0, 435, 0, 437, 438, 439, 440, 441, 0, + 442, 443, 444, 0, 0, 445, 446, 447, 448, 449, + 0, 450, 451, 452, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 471, 0, 472, 0, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, + 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, + 119, 120, 0, 0, 0, 0, 0, 0, 976, 0, + 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, + 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, + 139, 140, 141, 0, 0, 142, 143, 144, 145, 146, + 0, 147, 148, 149, 150, 151, 0, 0, 0, 153, + 154, 155, 156, 157, 158, 0, 160, 161, 162, 0, + 163, 164, 165, 166, 167, 168, 0, 0, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, + 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, + 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, + 0, 222, 0, 223, 0, 0, 0, 226, 227, 509, + 0, 230, 0, 231, 0, 232, 233, 234, 235, 0, + 236, 237, 238, 239, 240, 241, 242, 0, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 0, 257, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 0, 266, 0, 0, 269, 0, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 0, 279, + 0, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 0, 310, 311, 312, 313, 314, 0, 315, 316, 0, + 318, 0, 319, 320, 321, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 332, 0, 334, 0, 336, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 349, 0, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 511, 430, + 431, 432, 0, 433, 434, 0, 435, 0, 437, 438, + 439, 440, 441, 0, 442, 443, 444, 0, 0, 445, + 446, 447, 448, 449, 0, 450, 451, 452, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 471, 0, 472, 0, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, - 116, 117, 636, 118, 119, 120, 0, 0, 0, 0, + 116, 117, 639, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, @@ -9962,136 +10025,329 @@ static const yytype_int16 yytable[] = 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, - 0, 226, 227, 506, 0, 230, 0, 231, 0, 232, - 233, 234, 0, 235, 236, 237, 238, 239, 240, 241, - 0, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 0, 256, 257, 258, 259, - 260, 261, 262, 263, 0, 264, 0, 265, 0, 0, - 268, 0, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 0, 278, 0, 0, 280, 281, 282, 283, 284, - 285, 286, 287, 507, 289, 290, 291, 292, 293, 294, + 0, 226, 227, 509, 0, 230, 0, 231, 0, 232, + 233, 234, 235, 0, 236, 237, 238, 239, 240, 241, + 242, 0, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 0, 257, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 266, 0, + 0, 269, 0, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 0, 279, 0, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 0, 309, 310, 311, 312, 313, 0, - 314, 315, 0, 317, 0, 318, 319, 320, 321, 322, - 323, 0, 324, 325, 0, 0, 326, 327, 328, 0, - 0, 329, 330, 0, 332, 0, 334, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 0, 0, 0, - 0, 345, 346, 347, 0, 349, 350, 351, 352, 353, - 354, 0, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 364, 365, 366, 367, 368, 0, 369, 370, + 305, 306, 307, 308, 0, 310, 311, 312, 313, 314, + 0, 315, 316, 0, 318, 0, 319, 320, 321, 322, + 323, 324, 0, 325, 326, 0, 0, 327, 328, 329, + 0, 0, 330, 331, 332, 0, 334, 0, 336, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 349, 0, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 0, 385, 386, 387, 388, 389, + 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 0, 0, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, 0, 411, 412, 413, 414, 0, 415, 416, + 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 508, 428, 429, 430, 0, 431, 432, 0, 433, 0, - 435, 436, 437, 438, 439, 0, 640, 441, 0, 0, - 641, 443, 444, 445, 446, 0, 447, 448, 449, 450, - 451, 452, 453, 454, 0, 0, 455, 456, 457, 0, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 467, 468, 0, 469, 0, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 0, 477, 478, 479, 480, 481, + 427, 428, 511, 430, 431, 432, 0, 433, 434, 0, + 435, 0, 437, 438, 439, 440, 441, 0, 442, 643, + 444, 0, 0, 644, 446, 447, 448, 449, 0, 450, + 451, 452, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 471, 0, 472, 0, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 505, 0, 527, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, - 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, - 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, - 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, - 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, - 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, - 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, - 0, 0, 226, 227, 506, 0, 230, 0, 231, 0, - 232, 233, 234, 0, 235, 236, 237, 238, 239, 1316, - 241, 0, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 0, 256, 257, 258, - 259, 260, 261, 262, 263, 0, 264, 0, 265, 0, - 0, 268, 0, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 0, 278, 0, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 507, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 0, 309, 310, 311, 312, 313, - 0, 314, 315, 0, 317, 0, 318, 319, 320, 321, - 322, 323, 0, 324, 325, 0, 0, 326, 327, 328, - 0, 0, 329, 330, 0, 332, 0, 334, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 347, 0, 349, 350, 351, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 364, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 0, 385, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 0, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, 0, 411, 412, 413, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 508, 428, 429, 430, 0, 431, 432, 0, 433, - 0, 435, 436, 437, 438, 439, 0, 440, 441, 0, - 0, 442, 443, 444, 445, 446, 0, 447, 448, 449, - 450, 451, 452, 453, 454, 0, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 467, 468, 0, 469, 0, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 505, 0, 527, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 508, + 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, + 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, + 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, + 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, + 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, + 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, + 0, 223, 0, 0, 0, 226, 227, 509, 0, 230, + 0, 231, 0, 232, 233, 234, 235, 0, 236, 237, + 238, 239, 240, 1319, 242, 0, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 0, 257, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 266, 0, 0, 269, 0, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 0, 279, 0, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 0, 310, + 311, 312, 313, 314, 0, 315, 316, 0, 318, 0, + 319, 320, 321, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 332, 0, + 334, 0, 336, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 349, + 0, 351, 352, 353, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 366, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 0, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 0, 413, + 414, 415, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 511, 430, 431, 432, + 0, 433, 434, 0, 435, 0, 437, 438, 439, 440, + 441, 0, 442, 443, 444, 0, 0, 445, 446, 447, + 448, 449, 0, 450, 451, 452, 453, 454, 455, 456, + 457, 0, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 470, 471, 0, + 472, 0, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, 499, 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, - 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, - 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, - 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, - 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, - 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, - 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, - 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, - 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, - 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, - 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, + 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, + 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, + 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, + 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, + 145, 146, 0, 147, 148, 149, 150, 151, 0, 0, + 0, 153, 154, 155, 156, 157, 158, 0, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 0, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, + 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, + 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, + 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, + 220, 221, 0, 222, 0, 223, 0, 0, 0, 226, + 227, 509, 0, 230, 0, 231, 0, 232, 233, 234, + 235, 0, 236, 237, 238, 239, 240, 1321, 242, 0, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 0, 257, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 266, 0, 0, 269, + 0, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 0, 279, 0, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + 307, 308, 0, 310, 311, 312, 313, 314, 0, 315, + 316, 0, 318, 0, 319, 320, 321, 322, 323, 324, + 0, 325, 326, 0, 0, 327, 328, 329, 0, 0, + 330, 331, 332, 0, 334, 0, 336, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 349, 0, 351, 352, 353, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 366, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 0, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 0, 413, 414, 415, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 511, 430, 431, 432, 0, 433, 434, 0, 435, 0, + 437, 438, 439, 440, 441, 0, 442, 443, 444, 0, + 0, 445, 446, 447, 448, 449, 0, 450, 451, 452, + 453, 454, 455, 456, 457, 0, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 470, 471, 0, 472, 0, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, + 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 494, 495, 496, 497, 498, 499, 500, 508, 0, 530, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, + 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, + 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, + 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, + 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, + 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, + 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, + 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, + 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, + 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, + 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, + 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, - 0, 0, 0, 226, 227, 506, 0, 230, 0, 231, - 0, 232, 233, 234, 0, 235, 236, 237, 238, 239, - 1318, 241, 0, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 0, 256, 257, - 258, 259, 260, 261, 262, 263, 0, 264, 0, 265, - 0, 0, 268, 0, 270, 271, 272, 273, 274, 275, - 0, 0, 276, 0, 278, 0, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 507, 289, 290, 291, 292, + 0, 0, 0, 226, 227, 509, 0, 230, 0, 231, + 0, 232, 233, 234, 235, 0, 236, 237, 238, 239, + 240, 1324, 242, 0, 244, 245, 246, 247, 0, 248, + 249, 250, 251, 252, 253, 254, 0, 255, 0, 257, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, + 266, 0, 0, 269, 0, 271, 272, 273, 274, 275, + 276, 0, 0, 277, 0, 279, 0, 0, 281, 282, + 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 0, 309, 310, 311, 312, - 313, 0, 314, 315, 0, 317, 0, 318, 319, 320, - 321, 322, 323, 0, 324, 325, 0, 0, 326, 327, - 328, 0, 0, 329, 330, 0, 332, 0, 334, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 347, 0, 349, 350, 351, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 364, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 0, 385, 386, 387, + 303, 304, 305, 306, 307, 308, 0, 310, 311, 312, + 313, 314, 0, 315, 316, 0, 318, 0, 319, 320, + 321, 322, 323, 324, 0, 325, 326, 0, 0, 327, + 328, 329, 0, 0, 330, 331, 332, 0, 334, 0, + 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 0, 0, 0, 0, 347, 348, 349, 0, 351, + 352, 353, 354, 355, 356, 0, 357, 358, 359, 360, + 361, 362, 0, 363, 364, 365, 366, 367, 368, 369, + 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 0, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 0, 411, 412, 413, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 508, 428, 429, 430, 0, 431, 432, 0, - 433, 0, 435, 436, 437, 438, 439, 0, 440, 441, - 0, 0, 442, 443, 444, 445, 446, 0, 447, 448, - 449, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 467, 468, 0, 469, 0, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, + 398, 399, 400, 401, 0, 0, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 0, 413, 414, 415, + 416, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 511, 430, 431, 432, 0, 433, + 434, 0, 435, 0, 437, 438, 439, 440, 441, 0, + 442, 443, 444, 0, 0, 445, 446, 447, 448, 449, + 0, 450, 451, 452, 453, 454, 455, 456, 457, 0, + 0, 458, 459, 460, 0, 461, 462, 463, 464, 0, + 465, 466, 467, 468, 469, 470, 471, 0, 472, 0, + 474, 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 505, 0, - 527, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, + 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 500, 508, 0, 530, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, + 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, + 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, + 139, 140, 141, 0, 0, 142, 143, 144, 145, 146, + 0, 147, 148, 149, 150, 151, 0, 0, 0, 153, + 154, 155, 156, 157, 158, 0, 160, 161, 162, 0, + 163, 164, 165, 166, 167, 168, 0, 0, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, + 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, + 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, + 0, 222, 0, 223, 0, 0, 0, 226, 227, 509, + 0, 230, 0, 231, 0, 232, 233, 234, 235, 0, + 236, 237, 238, 239, 240, 1326, 242, 0, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 0, 257, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 0, 266, 0, 0, 269, 0, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 0, 279, + 0, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 0, 310, 311, 312, 313, 314, 0, 315, 316, 0, + 318, 0, 319, 320, 321, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 332, 0, 334, 0, 336, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 349, 0, 351, 352, 353, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 366, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 0, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 0, 413, 414, 415, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 511, 430, + 431, 432, 0, 433, 434, 0, 435, 0, 437, 438, + 439, 440, 441, 0, 442, 443, 444, 0, 0, 445, + 446, 447, 448, 449, 0, 450, 451, 452, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 470, + 471, 0, 472, 0, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, 499, 500, 508, 0, 530, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, + 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, + 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, + 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, + 143, 144, 145, 146, 0, 147, 148, 149, 150, 151, + 0, 0, 0, 153, 154, 155, 156, 157, 158, 0, + 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, + 0, 0, 170, 171, 172, 173, 174, 175, 176, 177, + 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, + 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, + 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, + 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, + 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, + 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, + 0, 226, 227, 509, 0, 230, 0, 231, 0, 232, + 233, 234, 235, 0, 236, 237, 238, 239, 240, 2162, + 242, 0, 244, 245, 246, 247, 0, 248, 249, 250, + 251, 252, 253, 254, 0, 255, 0, 257, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 266, 0, + 0, 269, 0, 271, 272, 273, 274, 275, 276, 0, + 0, 277, 0, 279, 0, 0, 281, 282, 283, 284, + 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 0, 310, 311, 312, 313, 314, + 0, 315, 316, 0, 318, 0, 319, 320, 321, 322, + 323, 324, 0, 325, 326, 0, 0, 327, 328, 329, + 0, 0, 330, 331, 332, 0, 334, 0, 336, 337, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 0, + 0, 0, 0, 347, 348, 349, 0, 351, 352, 353, + 354, 355, 356, 0, 357, 358, 359, 360, 361, 362, + 0, 363, 364, 365, 366, 367, 368, 369, 370, 0, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 0, 0, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, 0, 413, 414, 415, 416, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, + 427, 428, 511, 430, 431, 432, 0, 433, 434, 0, + 435, 0, 437, 438, 439, 440, 441, 0, 442, 443, + 444, 0, 0, 445, 446, 447, 448, 449, 0, 450, + 451, 452, 453, 454, 455, 456, 457, 0, 0, 458, + 459, 460, 0, 461, 462, 463, 464, 0, 465, 466, + 467, 468, 469, 470, 471, 0, 472, 0, 474, 475, + 476, 477, 478, 0, 0, 479, 0, 0, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 498, 499, 500, 751, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, + 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, + 0, 0, 0, 752, 0, 0, -764, 0, 753, 122, + 123, 0, 124, 125, 126, 754, 128, 129, 130, 0, + 755, 756, 757, 758, 0, 136, 137, 138, 139, 140, + 141, 0, 0, 142, 143, 759, 760, 146, 0, 147, + 148, 149, 150, 0, 0, 761, 0, 762, 154, 155, + 156, 157, 158, 763, 160, 161, 162, 0, 163, 164, + 165, 166, 167, 168, 0, 764, 170, 171, 172, 173, + 174, 175, 176, 177, 178, 179, 765, 766, 182, 0, + 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, + 191, 192, 193, 194, 0, 0, 195, 196, 767, 198, + 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, + 206, 207, 208, 209, 0, 211, 212, 213, 214, 215, + 0, 0, 217, 0, 218, 219, 768, 221, 0, 222, + 0, 223, 769, 0, 770, 226, 227, -764, 771, 230, + 0, 231, 0, 0, 0, 234, 235, 0, 236, 237, + 238, 239, 240, 241, 242, 773, 244, 245, 246, 247, + 0, 248, 249, 250, 251, 252, 253, 254, 0, 255, + 774, 0, 258, 259, 260, 261, 262, 775, 776, 0, + 777, 0, 266, 778, 779, 269, 780, 271, 272, 273, + 274, 275, 276, 0, 0, 277, 781, 279, 782, 0, + 281, 282, 283, 284, 285, 286, 287, 288, 783, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 784, 785, 786, + 311, 312, 313, 0, 0, 315, 316, 787, 318, 0, + 0, 320, 788, 322, 323, 324, 0, 325, 326, 0, + 0, 327, 328, 329, 0, 0, 330, 331, 0, 789, + 334, 790, 0, 337, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 0, 0, 0, 0, 347, 348, 0, + 791, 351, 352, 0, 354, 355, 356, 0, 357, 358, + 359, 360, 361, 362, 0, 363, 364, 365, 792, 367, + 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 793, 387, 388, 389, 794, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 0, 795, 402, 403, + 404, 405, 406, 407, 796, 409, 410, 411, 797, 413, + 414, 798, 416, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 426, 427, 428, 799, 430, 0, 432, + 0, 433, 434, 0, 435, 800, 437, 438, 439, 440, + 441, 0, 442, 801, 802, 0, 0, 445, 446, 0, + 448, 0, 0, 450, 451, 803, 453, 454, 455, 456, + 457, 804, 0, 458, 459, 460, 0, 461, 462, 463, + 464, 0, 465, 466, 467, 468, 469, 0, 805, 0, + 472, 806, 474, 475, 476, 477, 478, 0, 0, 479, + 0, 0, 480, 481, 482, 483, 484, 485, 508, 0, + 530, 0, 0, 0, 0, 0, 0, 0, 0, 497, + 498, 499, 500, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, @@ -10106,87 +10362,136 @@ static const yytype_int16 yytable[] = 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, - 223, 0, 0, 0, 226, 227, 506, 0, 230, 0, - 231, 0, 232, 233, 234, 0, 235, 236, 237, 238, - 239, 1321, 241, 0, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 0, 256, - 257, 258, 259, 260, 261, 262, 263, 0, 264, 0, - 265, 0, 0, 268, 0, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 0, 278, 0, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 507, 289, 290, 291, + 223, 0, 0, 0, 226, 227, 509, 0, 230, 0, + 231, 0, 232, 233, 234, 235, 0, 236, 237, 238, + 239, 240, 2872, 242, 0, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 0, + 257, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 266, 0, 0, 269, 0, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 0, 279, 0, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 0, 309, 310, 311, - 312, 313, 0, 314, 315, 0, 317, 0, 318, 319, - 320, 321, 322, 323, 0, 324, 325, 0, 0, 326, - 327, 328, 0, 0, 329, 330, 0, 332, 0, 334, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 347, 0, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 364, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 0, 385, 386, + 302, 303, 304, 305, 306, 307, 308, 0, 310, 311, + 312, 313, 314, 0, 315, 316, 0, 318, 0, 319, + 320, 321, 322, 323, 324, 0, 325, 326, 0, 0, + 327, 328, 329, 0, 0, 330, 331, 332, 0, 334, + 0, 336, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 349, 0, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 366, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 0, 411, 412, 413, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 508, 428, 429, 430, 0, 431, 432, - 0, 433, 0, 435, 436, 437, 438, 439, 0, 440, - 441, 0, 0, 442, 443, 444, 445, 446, 0, 447, - 448, 449, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 468, 0, 469, 0, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 505, - 0, 527, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, - 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, - 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, - 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, - 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, - 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, - 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, - 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, - 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, - 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, - 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, - 0, 223, 0, 0, 0, 226, 227, 506, 0, 230, - 0, 231, 0, 232, 233, 234, 0, 235, 236, 237, - 238, 239, 1323, 241, 0, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 0, - 256, 257, 258, 259, 260, 261, 262, 263, 0, 264, - 0, 265, 0, 0, 268, 0, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 0, 278, 0, 0, 280, - 281, 282, 283, 284, 285, 286, 287, 507, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 0, 309, 310, - 311, 312, 313, 0, 314, 315, 0, 317, 0, 318, - 319, 320, 321, 322, 323, 0, 324, 325, 0, 0, - 326, 327, 328, 0, 0, 329, 330, 0, 332, 0, - 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 347, 0, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 364, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 0, 385, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 0, 0, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 0, 411, 412, 413, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 508, 428, 429, 430, 0, 431, - 432, 0, 433, 0, 435, 436, 437, 438, 439, 0, - 440, 441, 0, 0, 442, 443, 444, 445, 446, 0, - 447, 448, 449, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 467, 468, 0, 469, 0, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 505, 0, 527, 0, 0, 0, 0, 0, 0, 0, + 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 0, 413, 414, + 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 511, 430, 431, 432, 0, + 433, 434, 0, 435, 0, 437, 438, 439, 440, 441, + 0, 442, 443, 444, 0, 0, 445, 446, 447, 448, + 449, 0, 450, 451, 452, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 470, 471, 0, 472, + 0, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, + 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, + 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, + 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, + 146, 0, 147, 148, 149, 150, 151, 0, 0, 0, + 153, 154, 155, 156, 157, 158, 0, 160, 161, 162, + 0, 163, 164, 165, 166, 167, 168, 0, 0, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, + 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, + 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, + 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, + 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, + 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, + 221, 0, 222, 0, 223, 0, 0, 0, 226, 227, + 509, 0, 230, 0, 231, 0, 232, 233, 234, 235, + 0, 236, 237, 238, 239, 240, 241, 242, 0, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 0, 257, 258, 259, 260, 261, 262, + 263, 264, 0, 265, 0, 266, 0, 0, 269, 0, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 0, + 279, 0, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 0, 310, 311, 312, 313, 314, 0, 315, 316, + 0, 318, 0, 319, 320, 321, 322, 323, 324, 0, + 325, 326, 0, 0, 327, 328, 329, 0, 0, 330, + 331, 332, 0, 334, 0, 336, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 349, 0, 351, 352, 353, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 366, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 0, 413, 414, 415, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 511, + 430, 431, 432, 0, 433, 434, 0, 435, 0, 437, + 438, 439, 440, 441, 0, 442, 443, 444, 0, 0, + 445, 446, 447, 448, 449, 0, 450, 451, 452, 453, + 454, 455, 456, 457, 0, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 470, 471, 0, 472, 0, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, 499, 500, 508, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, + 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, + 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, + 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, + 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, + 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, + 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, + 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, + 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, + 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, + 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, + 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, + 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, + 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, + 0, 0, 226, 227, 509, 0, 654, 0, 231, 0, + 232, 233, 234, 235, 0, 236, 237, 238, 239, 240, + 241, 242, 0, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 0, 257, 258, + 259, 260, 261, 262, 263, 264, 0, 265, 0, 266, + 0, 0, 269, 0, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 0, 279, 0, 0, 281, 282, 655, + 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 0, 310, 311, 312, 313, + 314, 0, 315, 316, 0, 318, 0, 319, 320, 321, + 322, 323, 324, 0, 325, 326, 0, 0, 327, 328, + 329, 0, 0, 330, 331, 332, 0, 334, 0, 336, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 349, 0, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 366, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, + 389, 390, 391, 392, 393, 394, 656, 396, 397, 398, + 399, 400, 401, 0, 0, 402, 403, 404, 405, 657, + 407, 408, 409, 410, 411, 0, 413, 414, 415, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 511, 430, 431, 432, 0, 433, 434, + 0, 435, 0, 437, 438, 439, 440, 441, 0, 442, + 658, 444, 0, 0, 445, 446, 447, 448, 449, 0, + 450, 451, 452, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 471, 0, 472, 0, 474, + 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -10202,137 +10507,91 @@ static const yytype_int16 yytable[] = 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, - 222, 0, 223, 0, 0, 0, 226, 227, 506, 0, - 230, 0, 231, 0, 232, 233, 234, 0, 235, 236, - 237, 238, 239, 2159, 241, 0, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 0, 256, 257, 258, 259, 260, 261, 262, 263, 0, - 264, 0, 265, 0, 0, 268, 0, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 0, 278, 0, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 507, 289, + 222, 0, 223, 0, 0, 0, 226, 227, 509, 0, + 230, 0, 231, 0, 232, 233, 234, 235, 0, 236, + 237, 238, 239, 240, 825, 242, 0, 244, 245, 246, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 0, 257, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 266, 0, 0, 269, 0, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 0, 279, 0, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 0, 309, - 310, 311, 312, 313, 0, 314, 315, 0, 317, 0, - 318, 319, 320, 321, 322, 323, 0, 324, 325, 0, - 0, 326, 327, 328, 0, 0, 329, 330, 0, 332, - 0, 334, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 347, 0, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 364, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 0, - 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 0, 411, 412, - 413, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 508, 428, 429, 430, 0, - 431, 432, 0, 433, 0, 435, 436, 437, 438, 439, - 0, 440, 441, 0, 0, 442, 443, 444, 445, 446, - 0, 447, 448, 449, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 468, 0, 469, 0, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 300, 301, 302, 303, 304, 305, 306, 307, 308, 0, + 310, 311, 312, 313, 314, 0, 315, 316, 0, 318, + 0, 319, 320, 321, 322, 323, 324, 0, 325, 326, + 0, 0, 327, 328, 329, 0, 0, 330, 331, 332, + 0, 334, 0, 336, 337, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 0, 0, 0, 0, 347, 348, + 349, 0, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 364, 365, 366, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, 0, + 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 511, 430, 431, + 432, 0, 433, 434, 0, 435, 0, 437, 438, 439, + 440, 441, 0, 442, 443, 444, 0, 0, 445, 446, + 447, 448, 449, 0, 450, 451, 452, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 0, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 470, 471, + 0, 472, 0, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 748, 0, 0, 0, 0, 0, 0, 0, 0, + 497, 498, 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, - 119, 120, 0, 0, 0, 749, 0, 0, -764, 0, - 750, 122, 123, 0, 124, 125, 126, 751, 128, 129, - 130, 0, 752, 753, 754, 755, 0, 136, 137, 138, - 139, 140, 141, 0, 0, 142, 143, 756, 757, 146, - 0, 147, 148, 149, 150, 0, 0, 758, 0, 759, - 154, 155, 156, 157, 158, 760, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 761, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 762, 763, - 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, - 764, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 0, 211, 212, 213, - 214, 215, 0, 0, 217, 0, 218, 219, 765, 221, - 0, 222, 0, 223, 766, 0, 767, 226, 227, -764, - 768, 230, 0, 231, 0, 0, 0, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 770, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 771, 0, 257, 258, 259, 260, 261, 772, 773, - 0, 774, 0, 265, 775, 776, 268, 777, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 778, 278, 779, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 780, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 781, 782, - 783, 310, 311, 312, 0, 0, 314, 315, 784, 317, - 0, 0, 319, 785, 321, 322, 323, 0, 324, 325, - 0, 0, 326, 327, 328, 0, 0, 329, 0, 786, - 332, 787, 0, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 0, - 788, 349, 350, 0, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 789, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 790, 385, 386, 387, 791, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 792, 400, 401, - 402, 403, 404, 405, 793, 407, 408, 409, 794, 411, - 412, 795, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 796, 428, 0, 430, - 0, 431, 432, 0, 433, 797, 435, 436, 437, 438, - 439, 0, 798, 799, 0, 0, 442, 443, 0, 445, - 0, 0, 447, 448, 800, 450, 451, 452, 453, 454, - 801, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 0, 802, 0, 469, - 803, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 505, 0, 527, - 0, 0, 0, 0, 0, 0, 0, 0, 494, 495, - 496, 497, 0, 0, 0, 0, 110, 111, 112, 113, - 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, - 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, - 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, - 0, 142, 143, 144, 145, 146, 0, 147, 148, 149, - 150, 151, 0, 0, 0, 153, 154, 155, 156, 157, - 158, 0, 160, 161, 162, 0, 163, 164, 165, 166, - 167, 168, 0, 0, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 0, 183, 0, - 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, - 193, 194, 0, 0, 195, 196, 197, 198, 199, 0, - 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, - 217, 0, 218, 219, 220, 221, 0, 222, 0, 223, - 0, 0, 0, 226, 227, 506, 0, 230, 0, 231, - 0, 232, 233, 234, 0, 235, 236, 237, 238, 239, - 2869, 241, 0, 243, 244, 245, 246, 0, 247, 248, - 249, 250, 251, 252, 253, 0, 254, 0, 256, 257, - 258, 259, 260, 261, 262, 263, 0, 264, 0, 265, - 0, 0, 268, 0, 270, 271, 272, 273, 274, 275, - 0, 0, 276, 0, 278, 0, 0, 280, 281, 282, - 283, 284, 285, 286, 287, 507, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 0, 309, 310, 311, 312, - 313, 0, 314, 315, 0, 317, 0, 318, 319, 320, - 321, 322, 323, 0, 324, 325, 0, 0, 326, 327, - 328, 0, 0, 329, 330, 0, 332, 0, 334, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 0, - 0, 0, 0, 345, 346, 347, 0, 349, 350, 351, - 352, 353, 354, 0, 355, 356, 357, 358, 359, 360, - 0, 361, 362, 363, 364, 365, 366, 367, 368, 0, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 0, 382, 383, 0, 385, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 0, 0, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 0, 411, 412, 413, 414, 0, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 508, 428, 429, 430, 0, 431, 432, 0, - 433, 0, 435, 436, 437, 438, 439, 0, 440, 441, - 0, 0, 442, 443, 444, 445, 446, 0, 447, 448, - 449, 450, 451, 452, 453, 454, 0, 0, 455, 456, - 457, 0, 458, 459, 460, 461, 0, 462, 463, 464, - 465, 466, 467, 468, 0, 469, 0, 471, 472, 473, - 474, 475, 0, 0, 476, 0, 0, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 505, 0, + 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, + 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, + 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, + 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, + 144, 145, 146, 0, 147, 148, 149, 150, 151, 0, + 0, 0, 153, 154, 155, 156, 157, 158, 0, 160, + 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, + 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, + 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, + 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, + 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, + 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, + 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, + 226, 227, 509, 0, 230, 0, 231, 0, 232, 233, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 0, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 0, 257, 258, 259, 260, + 261, 262, 263, 264, 0, 265, 0, 266, 0, 0, + 269, 0, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 0, 279, 0, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 0, 310, 311, 312, 313, 314, 0, + 315, 316, 0, 318, 0, 319, 320, 321, 322, 323, + 324, 0, 325, 326, 0, 0, 327, 328, 329, 0, + 0, 330, 331, 332, 0, 334, 0, 336, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 349, 0, 351, 352, 353, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 366, 367, 368, 369, 370, 0, 371, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, + 391, 392, 393, 394, 656, 396, 397, 398, 399, 400, + 401, 0, 0, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 0, 413, 414, 415, 416, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, + 428, 511, 430, 431, 432, 0, 433, 434, 0, 435, + 0, 437, 438, 439, 440, 441, 0, 442, 658, 444, + 0, 0, 445, 446, 447, 448, 449, 0, 450, 451, + 452, 453, 454, 455, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 470, 471, 0, 472, 0, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 498, 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, + 0, 0, 0, 0, 0, 1625, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, 148, @@ -10345,87 +10604,136 @@ static const yytype_int16 yytable[] = 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, 0, - 223, 0, 0, 0, 226, 227, 506, 0, 230, 0, - 231, 0, 232, 233, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 0, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 0, 256, - 257, 258, 259, 260, 261, 262, 263, 0, 264, 0, - 265, 0, 0, 268, 0, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 0, 278, 0, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 507, 289, 290, 291, + 223, 0, 0, 0, 226, 227, 509, 0, 230, 0, + 231, 0, 232, 233, 234, 235, 0, 236, 237, 238, + 239, 240, 241, 242, 0, 244, 245, 246, 247, 0, + 248, 249, 250, 251, 252, 253, 254, 0, 255, 0, + 257, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 266, 0, 0, 269, 0, 271, 272, 273, 274, + 275, 276, 0, 0, 277, 0, 279, 0, 0, 281, + 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 0, 309, 310, 311, - 312, 313, 0, 314, 315, 0, 317, 0, 318, 319, - 320, 321, 322, 323, 0, 324, 325, 0, 0, 326, - 327, 328, 0, 0, 329, 330, 0, 332, 0, 334, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 347, 0, 349, 350, - 351, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 364, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 0, 385, 386, + 302, 303, 304, 305, 306, 307, 308, 0, 310, 311, + 312, 313, 314, 0, 315, 316, 0, 318, 0, 319, + 320, 321, 322, 323, 324, 0, 325, 326, 0, 0, + 327, 328, 329, 0, 0, 330, 331, 332, 0, 334, + 0, 336, 337, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 0, 0, 0, 0, 347, 348, 349, 0, + 351, 352, 353, 354, 355, 356, 0, 357, 358, 359, + 360, 361, 362, 0, 363, 364, 365, 366, 367, 368, + 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 0, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 0, 411, 412, 413, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 508, 428, 429, 430, 0, 431, 432, - 0, 433, 0, 435, 436, 437, 438, 439, 0, 440, - 441, 0, 0, 442, 443, 444, 445, 446, 0, 447, - 448, 449, 450, 451, 452, 453, 454, 0, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 467, 468, 0, 469, 0, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 505, + 397, 398, 399, 400, 401, 0, 0, 402, 403, 404, + 405, 406, 0, 408, 409, 410, 411, 0, 413, 414, + 415, 416, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 426, 427, 428, 511, 430, 431, 432, 0, + 433, 434, 0, 435, 0, 437, 438, 439, 440, 441, + 0, 442, 443, 444, 0, 0, 445, 446, 447, 448, + 449, 0, 450, 451, 452, 453, 454, 455, 456, 457, + 0, 0, 458, 459, 460, 0, 461, 462, 463, 464, + 0, 465, 466, 467, 468, 469, 470, 471, 0, 472, + 0, 474, 475, 476, 477, 478, 0, 0, 479, 0, + 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 499, 500, 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, - 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, - 123, 0, 124, 125, 126, 0, 128, 129, 130, 131, - 132, 0, 134, 135, 0, 136, 137, 138, 139, 140, - 141, 0, 0, 142, 143, 144, 145, 146, 0, 147, - 148, 149, 150, 151, 0, 0, 0, 153, 154, 155, - 156, 157, 158, 0, 160, 161, 162, 0, 163, 164, - 165, 166, 167, 168, 0, 0, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 0, - 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, - 191, 192, 193, 194, 0, 0, 195, 196, 197, 198, - 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, - 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, 0, 217, 0, 218, 219, 220, 221, 0, 222, - 0, 223, 0, 0, 0, 226, 227, 506, 0, 651, - 0, 231, 0, 232, 233, 234, 0, 235, 236, 237, - 238, 239, 240, 241, 0, 243, 244, 245, 246, 0, - 247, 248, 249, 250, 251, 252, 253, 0, 254, 0, - 256, 257, 258, 259, 260, 261, 262, 263, 0, 264, - 0, 265, 0, 0, 268, 0, 270, 271, 272, 273, - 274, 275, 0, 0, 276, 0, 278, 0, 0, 280, - 281, 652, 283, 284, 285, 286, 287, 507, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 0, 309, 310, - 311, 312, 313, 0, 314, 315, 0, 317, 0, 318, - 319, 320, 321, 322, 323, 0, 324, 325, 0, 0, - 326, 327, 328, 0, 0, 329, 330, 0, 332, 0, - 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 0, 0, 0, 0, 345, 346, 347, 0, 349, - 350, 351, 352, 353, 354, 0, 355, 356, 357, 358, - 359, 360, 0, 361, 362, 363, 364, 365, 366, 367, - 368, 0, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 0, 382, 383, 0, 385, - 386, 387, 388, 389, 390, 391, 392, 653, 394, 395, - 396, 397, 398, 399, 0, 0, 400, 401, 402, 403, - 654, 405, 406, 407, 408, 409, 0, 411, 412, 413, - 414, 0, 415, 416, 417, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 508, 428, 429, 430, 0, 431, - 432, 0, 433, 0, 435, 436, 437, 438, 439, 0, - 655, 441, 0, 0, 442, 443, 444, 445, 446, 0, - 447, 448, 449, 450, 451, 452, 453, 454, 0, 0, - 455, 456, 457, 0, 458, 459, 460, 461, 0, 462, - 463, 464, 465, 466, 467, 468, 0, 469, 0, 471, - 472, 473, 474, 475, 0, 0, 476, 0, 0, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 505, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, + 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, + 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, + 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, + 146, 0, 147, 148, 149, 150, 151, 0, 0, 0, + 153, 154, 155, 156, 157, 158, 0, 160, 161, 162, + 0, 163, 164, 165, 166, 167, 168, 0, 0, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, + 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, + 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, + 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, + 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, + 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, + 221, 0, 222, 0, 223, 0, 0, 0, 226, 227, + 509, 0, 230, 0, 231, 0, 232, 233, 234, 235, + 0, 236, 237, 238, 239, 240, 1792, 242, 0, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 0, 257, 258, 259, 260, 261, 262, + 263, 264, 0, 265, 0, 266, 0, 0, 269, 0, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 0, + 279, 0, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 0, 310, 311, 312, 313, 314, 0, 315, 316, + 0, 318, 0, 319, 320, 321, 322, 323, 324, 0, + 325, 326, 0, 0, 327, 328, 329, 0, 0, 330, + 331, 332, 0, 334, 0, 336, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 349, 0, 351, 352, 353, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 366, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 0, 387, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 0, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 0, 413, 414, 415, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 511, + 430, 431, 432, 0, 433, 434, 0, 435, 0, 437, + 438, 439, 440, 441, 0, 442, 443, 444, 0, 0, + 445, 446, 447, 448, 449, 0, 450, 451, 452, 453, + 454, 455, 456, 457, 0, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 470, 471, 0, 472, 0, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, 499, 500, 508, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, + 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, + 125, 126, 0, 128, 129, 130, 131, 132, 0, 134, + 135, 0, 136, 137, 138, 139, 140, 141, 0, 0, + 142, 143, 144, 145, 146, 0, 147, 148, 149, 150, + 151, 0, 0, 0, 153, 154, 155, 156, 157, 158, + 0, 160, 161, 162, 0, 163, 164, 165, 166, 167, + 168, 0, 0, 170, 171, 172, 173, 174, 175, 176, + 177, 178, 179, 180, 181, 182, 0, 183, 0, 184, + 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, + 194, 0, 0, 195, 196, 197, 198, 199, 0, 200, + 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, + 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, + 0, 218, 219, 220, 221, 0, 222, 0, 223, 0, + 0, 0, 226, 227, 509, 0, 230, 0, 231, 0, + 232, 233, 234, 235, 0, 236, 237, 238, 239, 240, + 2149, 242, 0, 244, 245, 246, 247, 0, 248, 249, + 250, 251, 252, 253, 254, 0, 255, 0, 257, 258, + 259, 260, 261, 262, 263, 264, 0, 265, 0, 266, + 0, 0, 269, 0, 271, 272, 273, 274, 275, 276, + 0, 0, 277, 0, 279, 0, 0, 281, 282, 283, + 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 0, 310, 311, 312, 313, + 314, 0, 315, 316, 0, 318, 0, 319, 320, 321, + 322, 323, 324, 0, 325, 326, 0, 0, 327, 328, + 329, 0, 0, 330, 331, 332, 0, 334, 0, 336, + 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 0, 0, 0, 0, 347, 348, 349, 0, 351, 352, + 353, 354, 355, 356, 0, 357, 358, 359, 360, 361, + 362, 0, 363, 364, 365, 366, 367, 368, 369, 370, + 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 0, 384, 385, 0, 387, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 0, 0, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 0, 413, 414, 415, 416, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 511, 430, 431, 432, 0, 433, 434, + 0, 435, 0, 437, 438, 439, 440, 441, 0, 442, + 443, 444, 0, 0, 445, 446, 447, 448, 449, 0, + 450, 451, 452, 453, 454, 455, 456, 457, 0, 0, + 458, 459, 460, 0, 461, 462, 463, 464, 0, 465, + 466, 467, 468, 469, 470, 471, 0, 472, 0, 474, + 475, 476, 477, 478, 0, 0, 479, 0, 0, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, + 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -10441,467 +10749,228 @@ static const yytype_int16 yytable[] = 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, - 222, 0, 223, 0, 0, 0, 226, 227, 506, 0, - 230, 0, 231, 0, 232, 233, 234, 0, 235, 236, - 237, 238, 239, 822, 241, 0, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 0, 256, 257, 258, 259, 260, 261, 262, 263, 0, - 264, 0, 265, 0, 0, 268, 0, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 0, 278, 0, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 507, 289, + 222, 0, 223, 0, 0, 0, 226, 227, 509, 0, + 230, 0, 231, 0, 232, 233, 234, 235, 0, 236, + 237, 238, 239, 240, 2164, 242, 0, 244, 245, 246, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 0, 257, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 266, 0, 0, 269, 0, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 0, 279, 0, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 0, 309, - 310, 311, 312, 313, 0, 314, 315, 0, 317, 0, - 318, 319, 320, 321, 322, 323, 0, 324, 325, 0, - 0, 326, 327, 328, 0, 0, 329, 330, 0, 332, - 0, 334, 335, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 347, 0, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 362, 363, 364, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 0, - 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 0, 411, 412, - 413, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 508, 428, 429, 430, 0, - 431, 432, 0, 433, 0, 435, 436, 437, 438, 439, - 0, 440, 441, 0, 0, 442, 443, 444, 445, 446, - 0, 447, 448, 449, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 468, 0, 469, 0, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 300, 301, 302, 303, 304, 305, 306, 307, 308, 0, + 310, 311, 312, 313, 314, 0, 315, 316, 0, 318, + 0, 319, 320, 321, 322, 323, 324, 0, 325, 326, + 0, 0, 327, 328, 329, 0, 0, 330, 331, 332, + 0, 334, 0, 336, 337, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 0, 0, 0, 0, 347, 348, + 349, 0, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 364, 365, 366, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 0, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, 0, + 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 511, 430, 431, + 432, 0, 433, 434, 0, 435, 0, 437, 438, 439, + 440, 441, 0, 442, 443, 444, 0, 0, 445, 446, + 447, 448, 449, 0, 450, 451, 452, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 0, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 470, 471, + 0, 472, 0, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 505, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, - 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, - 130, 131, 132, 0, 134, 135, 0, 136, 137, 138, - 139, 140, 141, 0, 0, 142, 143, 144, 145, 146, - 0, 147, 148, 149, 150, 151, 0, 0, 0, 153, - 154, 155, 156, 157, 158, 0, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 0, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, - 197, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, - 0, 222, 0, 223, 0, 0, 0, 226, 227, 506, - 0, 230, 0, 231, 0, 232, 233, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 0, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 0, 256, 257, 258, 259, 260, 261, 262, 263, - 0, 264, 0, 265, 0, 0, 268, 0, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 0, 278, 0, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 507, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 0, - 309, 310, 311, 312, 313, 0, 314, 315, 0, 317, - 0, 318, 319, 320, 321, 322, 323, 0, 324, 325, - 0, 0, 326, 327, 328, 0, 0, 329, 330, 0, - 332, 0, 334, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 347, - 0, 349, 350, 351, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 364, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 0, 385, 386, 387, 388, 389, 390, 391, 392, 653, - 394, 395, 396, 397, 398, 399, 0, 0, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, 0, 411, - 412, 413, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 508, 428, 429, 430, - 0, 431, 432, 0, 433, 0, 435, 436, 437, 438, - 439, 0, 655, 441, 0, 0, 442, 443, 444, 445, - 446, 0, 447, 448, 449, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 467, 468, 0, 469, - 0, 471, 472, 473, 474, 475, 0, 0, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 505, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 0, 0, 0, 1622, - 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, - 129, 130, 131, 132, 0, 134, 135, 0, 136, 137, - 138, 139, 140, 141, 0, 0, 142, 143, 144, 145, - 146, 0, 147, 148, 149, 150, 151, 0, 0, 0, - 153, 154, 155, 156, 157, 158, 0, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 0, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 0, 183, 0, 184, 185, 186, 187, 188, - 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 197, 198, 199, 0, 200, 201, 202, 0, 203, - 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, - 221, 0, 222, 0, 223, 0, 0, 0, 226, 227, - 506, 0, 230, 0, 231, 0, 232, 233, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 0, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 0, 256, 257, 258, 259, 260, 261, 262, - 263, 0, 264, 0, 265, 0, 0, 268, 0, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 0, 278, - 0, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 507, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 0, 309, 310, 311, 312, 313, 0, 314, 315, 0, - 317, 0, 318, 319, 320, 321, 322, 323, 0, 324, - 325, 0, 0, 326, 327, 328, 0, 0, 329, 330, - 0, 332, 0, 334, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 347, 0, 349, 350, 351, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 364, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 0, 385, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 0, 400, - 401, 402, 403, 404, 0, 406, 407, 408, 409, 0, - 411, 412, 413, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 508, 428, 429, - 430, 0, 431, 432, 0, 433, 0, 435, 436, 437, - 438, 439, 0, 440, 441, 0, 0, 442, 443, 444, - 445, 446, 0, 447, 448, 449, 450, 451, 452, 453, - 454, 0, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 467, 468, 0, - 469, 0, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 505, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, - 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, - 128, 129, 130, 131, 132, 0, 134, 135, 0, 136, - 137, 138, 139, 140, 141, 0, 0, 142, 143, 144, - 145, 146, 0, 147, 148, 149, 150, 151, 0, 0, - 0, 153, 154, 155, 156, 157, 158, 0, 160, 161, - 162, 0, 163, 164, 165, 166, 167, 168, 0, 0, - 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 0, 183, 0, 184, 185, 186, 187, - 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, - 195, 196, 197, 198, 199, 0, 200, 201, 202, 0, - 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, - 220, 221, 0, 222, 0, 223, 0, 0, 0, 226, - 227, 506, 0, 230, 0, 231, 0, 232, 233, 234, - 0, 235, 236, 237, 238, 239, 1789, 241, 0, 243, - 244, 245, 246, 0, 247, 248, 249, 250, 251, 252, - 253, 0, 254, 0, 256, 257, 258, 259, 260, 261, - 262, 263, 0, 264, 0, 265, 0, 0, 268, 0, - 270, 271, 272, 273, 274, 275, 0, 0, 276, 0, - 278, 0, 0, 280, 281, 282, 283, 284, 285, 286, - 287, 507, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 0, 309, 310, 311, 312, 313, 0, 314, 315, - 0, 317, 0, 318, 319, 320, 321, 322, 323, 0, - 324, 325, 0, 0, 326, 327, 328, 0, 0, 329, - 330, 0, 332, 0, 334, 335, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 0, 0, 0, 0, 345, - 346, 347, 0, 349, 350, 351, 352, 353, 354, 0, - 355, 356, 357, 358, 359, 360, 0, 361, 362, 363, - 364, 365, 366, 367, 368, 0, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 0, - 382, 383, 0, 385, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 0, 0, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 0, 411, 412, 413, 414, 0, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 508, 428, - 429, 430, 0, 431, 432, 0, 433, 0, 435, 436, - 437, 438, 439, 0, 440, 441, 0, 0, 442, 443, - 444, 445, 446, 0, 447, 448, 449, 450, 451, 452, - 453, 454, 0, 0, 455, 456, 457, 0, 458, 459, - 460, 461, 0, 462, 463, 464, 465, 466, 467, 468, - 0, 469, 0, 471, 472, 473, 474, 475, 0, 0, - 476, 0, 0, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 505, 0, 0, 0, 0, 0, + 497, 498, 499, 500, 751, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, - 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, - 0, 128, 129, 130, 131, 132, 0, 134, 135, 0, + 117, 0, 118, 119, 120, 0, 0, 0, 752, 0, + 0, 0, 0, 753, 122, 123, 0, 124, 125, 126, + 754, 128, 129, 130, 0, 755, 756, 757, 758, 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, - 144, 145, 146, 0, 147, 148, 149, 150, 151, 0, - 0, 0, 153, 154, 155, 156, 157, 158, 0, 160, + 759, 760, 146, 0, 147, 148, 149, 150, 0, 0, + 761, 0, 762, 154, 155, 156, 157, 158, 763, 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, - 0, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 0, 183, 0, 184, 185, 186, + 764, 170, 171, 172, 173, 174, 175, 176, 177, 178, + 179, 765, 766, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 197, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, - 219, 220, 221, 0, 222, 0, 223, 0, 0, 0, - 226, 227, 506, 0, 230, 0, 231, 0, 232, 233, - 234, 0, 235, 236, 237, 238, 239, 2146, 241, 0, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 0, 256, 257, 258, 259, 260, - 261, 262, 263, 0, 264, 0, 265, 0, 0, 268, - 0, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 0, 278, 0, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 507, 289, 290, 291, 292, 293, 294, 295, + 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 0, + 211, 212, 213, 214, 215, 0, 0, 217, 0, 218, + 219, 768, 221, 0, 222, 0, 223, 769, 0, 770, + 226, 227, 0, 771, 230, 0, 231, 0, 0, 0, + 234, 235, 0, 236, 237, 238, 239, 240, 772, 242, + 773, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 774, 0, 258, 259, 260, + 261, 262, 775, 776, 0, 777, 0, 266, 778, 779, + 269, 780, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 781, 279, 782, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 783, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 0, 309, 310, 311, 312, 313, 0, 314, - 315, 0, 317, 0, 318, 319, 320, 321, 322, 323, - 0, 324, 325, 0, 0, 326, 327, 328, 0, 0, - 329, 330, 0, 332, 0, 334, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 347, 0, 349, 350, 351, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 364, 365, 366, 367, 368, 0, 369, 370, 371, + 306, 307, 784, 785, 786, 311, 312, 313, 0, 0, + 315, 316, 787, 318, 0, 0, 320, 788, 322, 323, + 324, 0, 325, 326, 0, 0, 327, 328, 329, 0, + 0, 330, 331, 0, 789, 334, 790, 0, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 0, 791, 351, 352, 0, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 792, 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 0, 385, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 0, 411, 412, 413, 414, 0, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 508, - 428, 429, 430, 0, 431, 432, 0, 433, 0, 435, - 436, 437, 438, 439, 0, 440, 441, 0, 0, 442, - 443, 444, 445, 446, 0, 447, 448, 449, 450, 451, - 452, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 467, - 468, 0, 469, 0, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 505, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, - 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, - 126, 0, 128, 129, 130, 131, 132, 0, 134, 135, - 0, 136, 137, 138, 139, 140, 141, 0, 0, 142, - 143, 144, 145, 146, 0, 147, 148, 149, 150, 151, - 0, 0, 0, 153, 154, 155, 156, 157, 158, 0, - 160, 161, 162, 0, 163, 164, 165, 166, 167, 168, - 0, 0, 170, 171, 172, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 0, 183, 0, 184, 185, - 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, - 0, 0, 195, 196, 197, 198, 199, 0, 200, 201, - 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, - 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, - 218, 219, 220, 221, 0, 222, 0, 223, 0, 0, - 0, 226, 227, 506, 0, 230, 0, 231, 0, 232, - 233, 234, 0, 235, 236, 237, 238, 239, 2161, 241, - 0, 243, 244, 245, 246, 0, 247, 248, 249, 250, - 251, 252, 253, 0, 254, 0, 256, 257, 258, 259, - 260, 261, 262, 263, 0, 264, 0, 265, 0, 0, - 268, 0, 270, 271, 272, 273, 274, 275, 0, 0, - 276, 0, 278, 0, 0, 280, 281, 282, 283, 284, - 285, 286, 287, 507, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 0, 309, 310, 311, 312, 313, 0, - 314, 315, 0, 317, 0, 318, 319, 320, 321, 322, - 323, 0, 324, 325, 0, 0, 326, 327, 328, 0, - 0, 329, 330, 0, 332, 0, 334, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 0, 0, 0, - 0, 345, 346, 347, 0, 349, 350, 351, 352, 353, - 354, 0, 355, 356, 357, 358, 359, 360, 0, 361, - 362, 363, 364, 365, 366, 367, 368, 0, 369, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 0, 382, 383, 0, 385, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 0, 0, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, 0, 411, 412, 413, 414, 0, 415, 416, - 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 508, 428, 429, 430, 0, 431, 432, 0, 433, 0, - 435, 436, 437, 438, 439, 0, 440, 441, 0, 0, - 442, 443, 444, 445, 446, 0, 447, 448, 449, 450, - 451, 452, 453, 454, 0, 0, 455, 456, 457, 0, - 458, 459, 460, 461, 0, 462, 463, 464, 465, 466, - 467, 468, 0, 469, 0, 471, 472, 473, 474, 475, - 0, 0, 476, 0, 0, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 748, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 110, 111, 112, 113, 114, - 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, - 749, 0, 0, 0, 0, 750, 122, 123, 0, 124, - 125, 126, 751, 128, 129, 130, 0, 752, 753, 754, - 755, 0, 136, 137, 138, 139, 140, 141, 0, 0, - 142, 143, 756, 757, 146, 0, 147, 148, 149, 150, - 0, 0, 758, 0, 759, 154, 155, 156, 157, 158, - 760, 160, 161, 162, 0, 163, 164, 165, 166, 167, - 168, 0, 761, 170, 171, 172, 173, 174, 175, 176, - 177, 178, 179, 762, 763, 182, 0, 183, 0, 184, - 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, - 194, 0, 0, 195, 196, 764, 198, 199, 0, 200, - 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, - 209, 0, 211, 212, 213, 214, 215, 0, 0, 217, - 0, 218, 219, 765, 221, 0, 222, 0, 223, 766, - 0, 767, 226, 227, 0, 768, 230, 0, 231, 0, - 0, 0, 234, 0, 235, 236, 237, 238, 239, 769, - 241, 770, 243, 244, 245, 246, 0, 247, 248, 249, - 250, 251, 252, 253, 0, 254, 771, 0, 257, 258, - 259, 260, 261, 772, 773, 0, 774, 0, 265, 775, - 776, 268, 777, 270, 271, 272, 273, 274, 275, 0, - 0, 276, 778, 278, 779, 0, 280, 281, 282, 283, - 284, 285, 286, 287, 780, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 781, 782, 783, 310, 311, 312, 0, - 0, 314, 315, 784, 317, 0, 0, 319, 785, 321, - 322, 323, 0, 324, 325, 0, 0, 326, 327, 328, - 0, 0, 329, 0, 786, 332, 787, 0, 335, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 0, 0, - 0, 0, 345, 346, 0, 788, 349, 350, 0, 352, - 353, 354, 0, 355, 356, 357, 358, 359, 360, 0, - 361, 362, 363, 789, 365, 366, 367, 368, 0, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 0, 382, 383, 790, 385, 386, 387, 791, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 0, 792, 400, 401, 402, 403, 404, 405, 793, - 407, 408, 409, 794, 411, 412, 795, 414, 0, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 796, 428, 0, 430, 0, 431, 432, 0, 433, - 797, 435, 436, 437, 438, 439, 0, 798, 799, 0, - 0, 442, 443, 0, 445, 0, 0, 447, 448, 800, - 450, 451, 452, 453, 454, 801, 0, 455, 456, 457, - 0, 458, 459, 460, 461, 0, 462, 463, 464, 465, - 466, 0, 802, 0, 469, 803, 471, 472, 473, 474, - 475, 0, 0, 476, 0, 0, 477, 478, 479, 480, - 481, 482, 748, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 494, 495, 496, 497, 0, 0, 0, + 382, 383, 0, 384, 385, 793, 387, 388, 389, 794, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 795, 402, 403, 404, 405, 406, 407, 796, + 409, 410, 411, 797, 413, 414, 798, 416, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, + 428, 799, 430, 0, 432, 0, 433, 434, 0, 435, + 800, 437, 438, 439, 440, 441, 0, 442, 801, 802, + 0, 0, 445, 446, 0, 448, 0, 0, 450, 451, + 803, 453, 454, 455, 456, 457, 804, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 0, 805, 0, 472, 806, 474, 475, 476, + 477, 478, 0, 0, 479, 0, 0, 480, 481, 482, + 483, 484, 485, 751, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 497, 498, 499, 500, 0, 0, + 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, + 0, 118, 119, 120, 0, 0, 0, 752, 0, 0, + 0, 0, 753, 122, 123, 0, 124, 125, 126, 754, + 128, 129, 130, 0, 755, 756, 757, 758, 0, 136, + 137, 138, 139, 140, 141, 0, 0, 142, 143, 759, + 760, 146, 0, 147, 148, 149, 150, 0, 0, 761, + 0, 762, 154, 155, 156, 157, 158, 763, 160, 161, + 162, 0, 163, 164, 165, 166, 167, 168, 0, 764, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 765, 766, 182, 0, 183, 0, 184, 185, 186, 187, + 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, + 195, 196, 767, 198, 199, 0, 200, 201, 202, 0, + 203, 204, 205, 0, 206, 207, 208, 209, 0, 211, + 212, 213, 214, 215, 0, 0, 217, 0, 218, 219, + 768, 221, 0, 222, 0, 223, 769, 0, 770, 226, + 227, 0, 771, 230, 0, 231, 0, 0, 0, 234, + 235, 0, 236, 237, 238, 239, 240, 241, 242, 773, + 244, 245, 246, 247, 0, 248, 249, 250, 251, 252, + 253, 254, 0, 255, 774, 0, 258, 259, 260, 261, + 262, 775, 776, 0, 777, 0, 266, 778, 779, 269, + 780, 271, 272, 273, 274, 275, 276, 0, 0, 277, + 781, 279, 782, 0, 281, 282, 283, 284, 285, 286, + 287, 288, 783, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + 307, 784, 785, 786, 311, 312, 313, 0, 0, 315, + 316, 787, 318, 0, 0, 320, 788, 322, 323, 324, + 0, 325, 326, 0, 0, 327, 328, 329, 0, 0, + 330, 331, 0, 789, 334, 790, 0, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 0, 0, 0, + 0, 347, 348, 0, 791, 351, 352, 0, 354, 355, + 356, 0, 357, 358, 359, 360, 361, 362, 0, 363, + 364, 365, 792, 367, 368, 369, 370, 0, 371, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 793, 387, 388, 389, 794, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 0, 795, 402, 403, 404, 405, 406, 407, 796, 409, + 410, 411, 797, 413, 414, 798, 416, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, + 799, 430, 0, 432, 0, 433, 434, 0, 435, 800, + 437, 438, 439, 440, 441, 0, 442, 801, 802, 0, + 0, 445, 446, 0, 448, 0, 0, 450, 451, 803, + 453, 454, 455, 456, 457, 804, 0, 458, 459, 460, + 0, 461, 462, 463, 464, 0, 465, 466, 467, 468, + 469, 0, 805, 0, 472, 806, 474, 475, 476, 477, + 478, 0, 0, 479, 0, 0, 480, 481, 482, 483, + 484, 485, 751, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 497, 498, 499, 500, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, - 118, 119, 120, 0, 0, 0, 749, 0, 0, 0, - 0, 750, 122, 123, 0, 124, 125, 126, 751, 128, - 129, 130, 0, 752, 753, 754, 755, 0, 136, 137, - 138, 139, 140, 141, 0, 0, 142, 143, 756, 757, - 146, 0, 147, 148, 149, 150, 0, 0, 758, 0, - 759, 154, 155, 156, 157, 158, 760, 160, 161, 162, - 0, 163, 164, 165, 166, 167, 168, 0, 761, 170, - 171, 172, 173, 174, 175, 176, 177, 178, 179, 762, - 763, 182, 0, 183, 0, 184, 185, 186, 187, 188, + 118, 119, 120, 0, 0, 0, 752, 0, 0, 0, + 0, 753, 122, 123, 0, 124, 125, 126, 754, 128, + 129, 130, 0, 755, 756, 757, 758, 0, 136, 137, + 138, 139, 140, 141, 0, 0, 142, 143, 759, 760, + 146, 0, 147, 148, 149, 150, 0, 0, 761, 0, + 762, 154, 155, 156, 157, 158, 763, 160, 161, 162, + 0, 163, 164, 165, 166, 167, 168, 0, 764, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 765, + 766, 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, 0, 195, - 196, 764, 198, 199, 0, 200, 201, 202, 0, 203, + 196, 767, 198, 199, 0, 200, 201, 202, 0, 1777, 204, 205, 0, 206, 207, 208, 209, 0, 211, 212, - 213, 214, 215, 0, 0, 217, 0, 218, 219, 765, - 221, 0, 222, 0, 223, 766, 0, 767, 226, 227, - 0, 768, 230, 0, 231, 0, 0, 0, 234, 0, - 235, 236, 237, 238, 239, 240, 241, 770, 243, 244, - 245, 246, 0, 247, 248, 249, 250, 251, 252, 253, - 0, 254, 771, 0, 257, 258, 259, 260, 261, 772, - 773, 0, 774, 0, 265, 775, 776, 268, 777, 270, - 271, 272, 273, 274, 275, 0, 0, 276, 778, 278, - 779, 0, 280, 281, 282, 283, 284, 285, 286, 287, - 780, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 781, - 782, 783, 310, 311, 312, 0, 0, 314, 315, 784, - 317, 0, 0, 319, 785, 321, 322, 323, 0, 324, - 325, 0, 0, 326, 327, 328, 0, 0, 329, 0, - 786, 332, 787, 0, 335, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 0, 0, 0, 0, 345, 346, - 0, 788, 349, 350, 0, 352, 353, 354, 0, 355, - 356, 357, 358, 359, 360, 0, 361, 362, 363, 789, - 365, 366, 367, 368, 0, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 0, 382, - 383, 790, 385, 386, 387, 791, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 0, 792, 400, - 401, 402, 403, 404, 405, 793, 407, 408, 409, 794, - 411, 412, 795, 414, 0, 415, 416, 417, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 796, 428, 0, - 430, 0, 431, 432, 0, 433, 797, 435, 436, 437, - 438, 439, 0, 798, 799, 0, 0, 442, 443, 0, - 445, 0, 0, 447, 448, 800, 450, 451, 452, 453, - 454, 801, 0, 455, 456, 457, 0, 458, 459, 460, - 461, 0, 462, 463, 464, 465, 466, 0, 802, 0, - 469, 803, 471, 472, 473, 474, 475, 0, 0, 476, - 0, 0, 477, 478, 479, 480, 481, 482, 748, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 494, - 495, 496, 497, 0, 0, 0, 0, 110, 111, 112, - 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, - 0, 0, 749, 0, 0, 0, 0, 750, 122, 123, - 0, 124, 125, 126, 751, 128, 129, 130, 0, 752, - 753, 754, 755, 0, 136, 137, 138, 139, 140, 141, - 0, 0, 142, 143, 756, 757, 146, 0, 147, 148, - 149, 150, 0, 0, 758, 0, 759, 154, 155, 156, - 157, 158, 760, 160, 161, 162, 0, 163, 164, 165, - 166, 167, 168, 0, 761, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 762, 763, 182, 0, 183, - 0, 184, 185, 186, 187, 188, 189, 0, 190, 191, - 192, 193, 194, 0, 0, 195, 196, 764, 198, 199, - 0, 200, 201, 202, 0, 1774, 204, 205, 0, 206, - 207, 208, 209, 0, 211, 212, 213, 214, 215, 0, - 0, 217, 0, 218, 219, 765, 221, 0, 222, 0, - 223, 766, 0, 767, 226, 227, 0, 768, 230, 0, - 231, 0, 0, 0, 234, 0, 235, 236, 237, 238, - 239, 240, 241, 770, 243, 244, 245, 246, 0, 247, - 248, 249, 250, 251, 252, 253, 0, 254, 771, 0, - 257, 258, 259, 260, 261, 772, 773, 0, 774, 0, - 265, 775, 776, 268, 777, 270, 271, 272, 273, 274, - 275, 0, 0, 276, 778, 278, 779, 0, 280, 281, - 282, 283, 284, 285, 286, 287, 780, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 781, 782, 783, 310, 311, - 312, 0, 0, 314, 315, 784, 317, 0, 0, 319, - 785, 321, 322, 323, 0, 324, 325, 0, 0, 326, - 327, 328, 0, 0, 329, 0, 786, 332, 787, 0, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 0, 0, 0, 0, 345, 346, 0, 788, 349, 350, - 0, 352, 353, 354, 0, 355, 356, 357, 358, 359, - 360, 0, 361, 362, 363, 789, 365, 366, 367, 368, - 0, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 0, 382, 383, 790, 385, 386, - 387, 791, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 0, 792, 400, 401, 402, 403, 404, - 405, 793, 407, 408, 409, 794, 411, 412, 795, 414, - 0, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 796, 428, 0, 430, 0, 431, 432, - 0, 433, 797, 435, 436, 437, 438, 439, 0, 798, - 799, 0, 0, 442, 443, 0, 445, 0, 0, 447, - 448, 800, 450, 451, 452, 453, 454, 801, 0, 455, - 456, 457, 0, 458, 459, 460, 461, 0, 462, 463, - 464, 465, 466, 0, 802, 0, 469, 803, 471, 472, - 473, 474, 475, 0, 0, 476, 0, 0, 477, 478, - 479, 480, 481, 482, 2795, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 494, 495, 496, 497, 0, - 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, - 117, 0, 118, 119, 120, 0, 0, 0, 2796, 0, - 0, 0, 0, 2797, 122, 123, 0, 124, 125, 126, - 2798, 128, 129, 130, 0, 752, 2799, 754, 755, 0, - 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, - 756, 757, 146, 0, 147, 148, 149, 150, 0, 0, - 2800, 0, 2801, 154, 155, 156, 157, 158, 2802, 160, - 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, - 2803, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 762, 763, 182, 0, 183, 0, 184, 185, 186, - 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, - 0, 195, 196, 764, 198, 199, 0, 200, 201, 202, - 0, 203, 204, 205, 0, 206, 207, 208, 209, 0, - 211, 212, 213, 214, 215, 0, 0, 217, 0, 218, - 219, 765, 221, 0, 222, 0, 223, 2804, 0, 2805, - 226, 227, 2806, 2807, 230, 0, 231, 0, 0, 0, - 234, 0, 235, 236, 237, 238, 239, 240, 241, 2808, - 243, 244, 245, 246, 0, 247, 248, 249, 250, 251, - 252, 253, 0, 254, 2809, 0, 257, 258, 259, 260, - 261, 772, 773, 0, 774, 0, 265, 2810, 2811, 268, - 2812, 270, 271, 272, 273, 274, 275, 0, 0, 276, - 2813, 278, 2814, 0, 280, 281, 282, 283, 284, 285, - 286, 287, 3033, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 781, 2816, 783, 310, 311, 312, 0, 0, 314, - 315, 2818, 317, 0, 0, 319, 785, 321, 322, 323, - 0, 324, 325, 0, 0, 326, 327, 328, 0, 0, - 329, 0, 2820, 332, 2821, 0, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 0, 0, 0, 0, - 345, 346, 0, 2822, 349, 350, 0, 352, 353, 354, - 0, 355, 356, 357, 358, 359, 360, 0, 361, 362, - 363, 789, 365, 366, 367, 368, 0, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 0, 382, 383, 2823, 385, 386, 387, 0, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 0, - 2824, 400, 401, 402, 403, 404, 405, 0, 407, 408, - 409, 2826, 411, 412, 795, 414, 0, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 3034, - 428, 0, 430, 0, 431, 432, 0, 433, 2828, 435, - 436, 437, 438, 439, 0, 798, 799, 0, 0, 442, - 443, 0, 445, 0, 0, 447, 448, 2829, 450, 451, - 452, 453, 454, 0, 0, 455, 456, 457, 0, 458, - 459, 460, 461, 0, 462, 463, 464, 465, 466, 0, - 802, 0, 469, 2831, 471, 472, 473, 474, 475, 0, - 0, 476, 0, 0, 477, 478, 479, 480, 481, 482, - 505, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 494, 495, 496, 497, 0, 0, 0, 0, 110, + 213, 214, 215, 0, 0, 217, 0, 218, 219, 768, + 221, 0, 222, 0, 223, 769, 0, 770, 226, 227, + 0, 771, 230, 0, 231, 0, 0, 0, 234, 235, + 0, 236, 237, 238, 239, 240, 241, 242, 773, 244, + 245, 246, 247, 0, 248, 249, 250, 251, 252, 253, + 254, 0, 255, 774, 0, 258, 259, 260, 261, 262, + 775, 776, 0, 777, 0, 266, 778, 779, 269, 780, + 271, 272, 273, 274, 275, 276, 0, 0, 277, 781, + 279, 782, 0, 281, 282, 283, 284, 285, 286, 287, + 288, 783, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 784, 785, 786, 311, 312, 313, 0, 0, 315, 316, + 787, 318, 0, 0, 320, 788, 322, 323, 324, 0, + 325, 326, 0, 0, 327, 328, 329, 0, 0, 330, + 331, 0, 789, 334, 790, 0, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 0, 0, 0, 0, + 347, 348, 0, 791, 351, 352, 0, 354, 355, 356, + 0, 357, 358, 359, 360, 361, 362, 0, 363, 364, + 365, 792, 367, 368, 369, 370, 0, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 793, 387, 388, 389, 794, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 0, + 795, 402, 403, 404, 405, 406, 407, 796, 409, 410, + 411, 797, 413, 414, 798, 416, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 799, + 430, 0, 432, 0, 433, 434, 0, 435, 800, 437, + 438, 439, 440, 441, 0, 442, 801, 802, 0, 0, + 445, 446, 0, 448, 0, 0, 450, 451, 803, 453, + 454, 455, 456, 457, 804, 0, 458, 459, 460, 0, + 461, 462, 463, 464, 0, 465, 466, 467, 468, 469, + 0, 805, 0, 472, 806, 474, 475, 476, 477, 478, + 0, 0, 479, 0, 0, 480, 481, 482, 483, 484, + 485, 2798, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 497, 498, 499, 500, 0, 0, 0, 0, + 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, + 119, 120, 0, 0, 0, 2799, 0, 0, 0, 0, + 2800, 122, 123, 0, 124, 125, 126, 2801, 128, 129, + 130, 0, 755, 2802, 757, 758, 0, 136, 137, 138, + 139, 140, 141, 0, 0, 142, 143, 759, 760, 146, + 0, 147, 148, 149, 150, 0, 0, 2803, 0, 2804, + 154, 155, 156, 157, 158, 2805, 160, 161, 162, 0, + 163, 164, 165, 166, 167, 168, 0, 2806, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 765, 766, + 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, + 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, + 767, 198, 199, 0, 200, 201, 202, 0, 203, 204, + 205, 0, 206, 207, 208, 209, 0, 211, 212, 213, + 214, 215, 0, 0, 217, 0, 218, 219, 768, 221, + 0, 222, 0, 223, 2807, 0, 2808, 226, 227, 2809, + 2810, 230, 0, 231, 0, 0, 0, 234, 235, 0, + 236, 237, 238, 239, 240, 241, 242, 2811, 244, 245, + 246, 247, 0, 248, 249, 250, 251, 252, 253, 254, + 0, 255, 2812, 0, 258, 259, 260, 261, 262, 775, + 776, 0, 777, 0, 266, 2813, 2814, 269, 2815, 271, + 272, 273, 274, 275, 276, 0, 0, 277, 2816, 279, + 2817, 0, 281, 282, 283, 284, 285, 286, 287, 288, + 3036, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, 307, 784, + 2819, 786, 311, 312, 313, 0, 0, 315, 316, 2821, + 318, 0, 0, 320, 788, 322, 323, 324, 0, 325, + 326, 0, 0, 327, 328, 329, 0, 0, 330, 331, + 0, 2823, 334, 2824, 0, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 0, 0, 0, 0, 347, + 348, 0, 2825, 351, 352, 0, 354, 355, 356, 0, + 357, 358, 359, 360, 361, 362, 0, 363, 364, 365, + 792, 367, 368, 369, 370, 0, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 0, + 384, 385, 2826, 387, 388, 389, 0, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 0, 2827, + 402, 403, 404, 405, 406, 407, 0, 409, 410, 411, + 2829, 413, 414, 798, 416, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 3037, 430, + 0, 432, 0, 433, 434, 0, 435, 2831, 437, 438, + 439, 440, 441, 0, 442, 801, 802, 0, 0, 445, + 446, 0, 448, 0, 0, 450, 451, 2832, 453, 454, + 455, 456, 457, 0, 0, 458, 459, 460, 0, 461, + 462, 463, 464, 0, 465, 466, 467, 468, 469, 0, + 805, 0, 472, 2834, 474, 475, 476, 477, 478, 0, + 0, 479, 0, 0, 480, 481, 482, 483, 484, 485, + 508, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 497, 498, 499, 500, 0, 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, 119, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, 0, 124, 125, 126, 0, 128, 129, 130, @@ -10916,1103 +10985,1063 @@ static const yytype_int16 yytable[] = 198, 199, 0, 200, 201, 202, 0, 203, 204, 205, 0, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 0, 217, 0, 218, 219, 220, 221, 0, - 222, 0, 223, 0, 0, 0, 226, 227, 506, 0, - 230, 0, 231, 0, 232, 233, 234, 0, 235, 236, - 237, 238, 239, 240, 241, 0, 243, 244, 245, 246, - 0, 247, 248, 249, 250, 251, 252, 253, 0, 254, - 0, 256, 257, 258, 259, 260, 261, 262, 263, 0, - 264, 0, 265, 0, 0, 268, 0, 270, 271, 272, - 273, 274, 275, 0, 0, 276, 0, 278, 0, 0, - 280, 281, 282, 283, 284, 285, 286, 287, 507, 289, + 222, 0, 223, 0, 0, 0, 226, 227, 509, 0, + 230, 0, 231, 0, 232, 233, 0, 235, 0, 236, + 237, 238, 239, 240, 241, 242, 0, 244, 245, 246, + 247, 0, 248, 249, 250, 251, 252, 253, 254, 0, + 255, 0, 257, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 266, 0, 0, 269, 0, 271, 272, + 273, 274, 275, 276, 0, 0, 277, 0, 279, 0, + 0, 281, 282, 283, 284, 285, 286, 287, 288, 510, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 0, 309, - 310, 311, 312, 313, 0, 314, 315, 0, 317, 0, - 318, 319, 320, 321, 322, 323, 0, 324, 325, 0, - 0, 326, 327, 328, 0, 0, 329, 330, 0, 332, - 0, 334, 335, 336, 337, 338, 339, 340, 0, 342, - 343, 344, 0, 0, 0, 0, 345, 346, 347, 0, - 349, 350, 351, 352, 353, 354, 0, 355, 356, 357, - 358, 359, 360, 0, 361, 0, 363, 364, 365, 366, - 367, 368, 0, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 0, 382, 383, 0, - 385, 386, 387, 388, 0, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 0, 0, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, 0, 411, 412, - 413, 414, 0, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 508, 428, 429, 430, 0, - 431, 432, 0, 433, 0, 435, 436, 437, 438, 439, - 0, 440, 441, 0, 0, 442, 443, 444, 445, 446, - 0, 447, 448, 449, 450, 451, 452, 453, 454, 0, - 0, 455, 456, 457, 0, 458, 459, 460, 461, 0, - 462, 463, 464, 465, 466, 467, 468, 0, 469, 0, - 471, 472, 473, 474, 475, 0, 0, 476, 0, 0, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 300, 301, 302, 303, 304, 305, 306, 307, 308, 0, + 310, 311, 312, 313, 314, 0, 315, 316, 0, 318, + 0, 319, 320, 321, 322, 323, 324, 0, 325, 326, + 0, 0, 327, 328, 329, 0, 0, 330, 331, 332, + 0, 334, 0, 336, 337, 338, 339, 340, 341, 342, + 0, 344, 345, 346, 0, 0, 0, 0, 347, 348, + 349, 0, 351, 352, 353, 354, 355, 356, 0, 357, + 358, 359, 360, 361, 362, 0, 363, 0, 365, 366, + 367, 368, 369, 370, 0, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 0, 387, 388, 389, 390, 0, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 0, 0, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, 0, + 413, 414, 415, 416, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 511, 430, 431, + 432, 0, 433, 434, 0, 435, 0, 437, 438, 439, + 440, 441, 0, 442, 443, 444, 0, 0, 445, 446, + 447, 448, 449, 0, 450, 451, 452, 453, 454, 455, + 456, 457, 0, 0, 458, 459, 460, 0, 461, 462, + 463, 464, 0, 465, 466, 467, 468, 469, 470, 471, + 0, 472, 0, 474, 475, 476, 477, 478, 0, 0, + 479, 0, 0, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 748, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 110, 111, 112, 113, 114, 115, 116, 117, 0, 118, - 119, 120, 0, 0, 0, 749, 0, 0, 0, 0, - 750, 122, 123, 0, 124, 125, 126, 751, 128, 129, - 130, 0, 752, 753, 754, 755, 0, 136, 137, 138, - 139, 140, 141, 0, 0, 142, 143, 756, 757, 146, - 0, 147, 148, 149, 150, 0, 0, 758, 0, 759, - 154, 155, 156, 157, 158, 760, 160, 161, 162, 0, - 163, 164, 165, 166, 167, 168, 0, 761, 170, 171, - 172, 173, 174, 175, 176, 177, 178, 179, 762, 763, - 182, 0, 183, 0, 184, 185, 186, 187, 188, 189, - 0, 190, 191, 192, 193, 194, 0, 0, 195, 196, - 764, 198, 199, 0, 200, 201, 202, 0, 203, 204, - 205, 0, 206, 207, 208, 209, 0, 211, 212, 213, - 214, 215, 0, 0, 217, 0, 218, 219, 765, 221, - 0, 222, 0, 223, 766, 0, 767, 226, 227, 0, - 768, 230, 0, 231, 0, 0, 0, 234, 0, 235, - 236, 237, 238, 239, 240, 241, 770, 243, 244, 245, - 246, 0, 247, 248, 249, 250, 251, 252, 253, 0, - 254, 771, 0, 257, 258, 259, 260, 261, 772, 773, - 0, 774, 0, 265, 775, 776, 268, 777, 270, 271, - 272, 273, 274, 275, 0, 0, 276, 778, 278, 779, - 0, 280, 281, 282, 283, 284, 285, 286, 287, 0, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 781, 782, - 783, 310, 311, 312, 0, 0, 314, 315, 784, 317, - 0, 0, 319, 785, 321, 322, 323, 0, 324, 325, - 0, 0, 326, 327, 328, 0, 0, 329, 0, 786, - 332, 787, 0, 335, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 0, 0, 0, 0, 345, 346, 0, - 788, 349, 350, 0, 352, 353, 354, 0, 355, 356, - 357, 358, 359, 360, 0, 361, 362, 363, 789, 365, - 366, 367, 368, 0, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 0, 382, 383, - 790, 385, 386, 387, 0, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 0, 792, 400, 401, - 402, 403, 404, 405, 0, 407, 408, 409, 794, 411, - 412, 795, 414, 0, 415, 416, 417, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 0, 428, 0, 430, - 0, 431, 432, 0, 433, 797, 435, 436, 437, 438, - 439, 0, 798, 799, 0, 0, 442, 443, 0, 445, - 0, 0, 447, 448, 800, 450, 451, 452, 453, 454, - 0, 0, 455, 456, 457, 0, 458, 459, 460, 461, - 0, 462, 463, 464, 465, 466, 0, 802, 0, 469, - 803, 471, 472, 473, 474, 475, 0, 1, 476, 0, - 0, 477, 478, 479, 480, 481, 482, 2, 0, 3, - 4, 0, 0, 0, 1, 0, 0, 0, 494, 495, - 496, 497, 0, 0, 2, 0, 0, 6, 0, 0, - 0, 0, 0, 0, 0, 0, 7, 0, 0, 0, - 0, 0, 0, 0, 6, 0, 0, 0, 8, 0, - 0, 0, 0, 7, 0, 0, 0, 0, 0, 9, - 0, 0, 0, 0, 0, 8, 0, 0, 0, 0, - 0, 10, 0, 11, 0, 0, 9, 0, 0, 0, - 0, 0, 0, 0, 12, 0, 0, 0, 10, 0, - 11, 0, 0, 0, 0, 0, 0, 0, 13, 0, - 0, 12, 0, 0, 0, 0, 0, 0, 15, 0, - 0, 0, 0, 0, 16, 13, 0, 0, 0, 0, - 0, 0, 17, 0, 0, 15, 0, 0, 0, 0, - 0, 16, 0, 0, 0, 0, 0, 0, 20, 17, - 0, 0, 21, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 20, 0, 0, 0, 21, + 497, 498, 499, 500, 751, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 110, 111, 112, 113, 114, 115, 116, + 117, 0, 118, 119, 120, 0, 0, 0, 752, 0, + 0, 0, 0, 753, 122, 123, 0, 124, 125, 126, + 754, 128, 129, 130, 0, 755, 756, 757, 758, 0, + 136, 137, 138, 139, 140, 141, 0, 0, 142, 143, + 759, 760, 146, 0, 147, 148, 149, 150, 0, 0, + 761, 0, 762, 154, 155, 156, 157, 158, 763, 160, + 161, 162, 0, 163, 164, 165, 166, 167, 168, 0, + 764, 170, 171, 172, 173, 174, 175, 176, 177, 178, + 179, 765, 766, 182, 0, 183, 0, 184, 185, 186, + 187, 188, 189, 0, 190, 191, 192, 193, 194, 0, + 0, 195, 196, 767, 198, 199, 0, 200, 201, 202, + 0, 203, 204, 205, 0, 206, 207, 208, 209, 0, + 211, 212, 213, 214, 215, 0, 0, 217, 0, 218, + 219, 768, 221, 0, 222, 0, 223, 769, 0, 770, + 226, 227, 0, 771, 230, 0, 231, 0, 0, 0, + 234, 235, 0, 236, 237, 238, 239, 240, 241, 242, + 773, 244, 245, 246, 247, 0, 248, 249, 250, 251, + 252, 253, 254, 0, 255, 774, 0, 258, 259, 260, + 261, 262, 775, 776, 0, 777, 0, 266, 778, 779, + 269, 780, 271, 272, 273, 274, 275, 276, 0, 0, + 277, 781, 279, 782, 0, 281, 282, 283, 284, 285, + 286, 287, 288, 0, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 784, 785, 786, 311, 312, 313, 0, 0, + 315, 316, 787, 318, 0, 0, 320, 788, 322, 323, + 324, 0, 325, 326, 0, 0, 327, 328, 329, 0, + 0, 330, 331, 0, 789, 334, 790, 0, 337, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 0, 0, + 0, 0, 347, 348, 0, 791, 351, 352, 0, 354, + 355, 356, 0, 357, 358, 359, 360, 361, 362, 0, + 363, 364, 365, 792, 367, 368, 369, 370, 0, 371, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 793, 387, 388, 389, 0, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 0, 795, 402, 403, 404, 405, 406, 407, 0, + 409, 410, 411, 797, 413, 414, 798, 416, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, + 428, 0, 430, 0, 432, 0, 433, 434, 0, 435, + 800, 437, 438, 439, 440, 441, 0, 442, 801, 802, + 0, 0, 445, 446, 0, 448, 0, 0, 450, 451, + 803, 453, 454, 455, 456, 457, 0, 0, 458, 459, + 460, 0, 461, 462, 463, 464, 0, 465, 466, 467, + 468, 469, 0, 805, 0, 472, 806, 474, 475, 476, + 477, 478, 0, 1, 479, 0, 0, 480, 481, 482, + 483, 484, 485, 2, 0, 3, 4, 0, 0, 0, + 1, 0, 0, 0, 497, 498, 499, 500, 0, 0, + 2, 0, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 7, 0, 0, 0, 0, 0, 0, 0, + 6, 0, 0, 0, 8, 0, 0, 0, 0, 7, + 0, 0, 0, 0, 0, 9, 0, 0, 0, 0, + 0, 8, 0, 0, 0, 0, 0, 10, 0, 11, + 0, 0, 9, 0, 0, 0, 0, 0, 0, 0, + 12, 0, 0, 0, 10, 0, 11, 0, 0, 0, + 0, 0, 0, 0, 13, 0, 0, 12, 0, 0, + 0, 0, 0, 0, 15, 0, 0, 0, 0, 0, + 16, 13, 0, 0, 0, 0, 0, 0, 17, 0, + 0, 15, 0, 0, 0, 0, 0, 16, 0, 0, + 0, 0, 0, 0, 20, 17, 0, 0, 21, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 23, 0, 0, 0, 0, 0, 0, + 0, 20, 0, 0, 0, 21, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 23, 0, 0, 0, 0, 0, 0, 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 23, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 23, 0, 0, 0, 0, 0, 0, 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 25, 26, 27, 0, 0, 0, 0, 0, 28, 0, - 0, 29, 0, 0, 0, 0, 0, 25, 26, 27, - 0, 0, 0, 0, 0, 28, 0, 0, 29, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 25, 26, + 27, 0, 0, 0, 0, 0, 28, 0, 0, 29, + 0, 0, 0, 0, 0, 25, 26, 27, 0, 0, + 0, 0, 0, 28, 0, 0, 29, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 30, 0, + 0, 0, 0, 0, 0, 0, 0, 31, 0, 0, + 0, 0, 0, 0, 0, 30, 0, 0, 0, 0, + 0, 32, 0, 0, 31, 0, 0, 0, 33, 0, + 0, 0, 34, 0, 0, 0, 0, 0, 32, 0, + 0, 0, 35, 0, 0, 33, 0, 0, 0, 34, + 0, 0, 0, 0, 36, 0, 0, 0, 37, 35, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 30, 0, 0, 0, 0, 0, 0, 0, 0, 31, - 0, 0, 0, 0, 0, 0, 0, 30, 0, 0, - 0, 0, 0, 32, 0, 0, 31, 0, 0, 0, - 33, 0, 0, 0, 34, 0, 0, 0, 0, 0, - 32, 0, 0, 0, 35, 0, 0, 33, 0, 0, - 0, 34, 0, 0, 0, 0, 36, 0, 0, 0, - 37, 35, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 36, 0, 0, 0, 37, 0, 38, + 0, 36, 0, 0, 0, 37, 0, 0, 38, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 39, 0, 0, 0, 38, 0, 41, 0, - 0, 0, 0, 42, 0, 0, 0, 562, 0, 39, - 0, 0, 0, 0, 0, 41, 0, 43, 0, 0, - 42, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 43, 0, 0, 0, 0, 0, - 0, 44, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 563, 0, 0, 44, 0, + 0, 39, 0, 0, 0, 38, 0, 41, 0, 0, + 0, 0, 42, 0, 0, 0, 565, 0, 39, 0, + 0, 0, 0, 0, 41, 0, 43, 0, 0, 42, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 43, 0, 0, 0, 0, 0, 0, + 44, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 566, 0, 0, 44, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 45 + 0, 45 }; static const yytype_int16 yycheck[] = { - 7, 498, 0, 0, 708, 45, 557, 0, 0, 69, - 0, 720, 793, 623, 21, 0, 1643, 672, 0, 940, - 920, 563, 1167, 7, 1180, 1457, 1604, 1142, 1103, 1505, - 555, 2092, 37, 0, 982, 1144, 18, 21, 36, 725, - 18, 1169, 831, 649, 73, 1530, 920, 1147, 716, 73, - 1344, 0, 7, 0, 799, 2102, 1136, 2157, 2071, 824, - 2073, 1297, 827, 1172, 71, 72, 21, 694, 931, 1349, - 1746, 931, 931, 555, 562, 2204, 2205, 2206, 2562, 708, - 563, 710, 931, 712, 558, 1557, 1558, 71, 72, 1070, - 2037, 2564, 0, 555, 1075, 619, 94, 2153, 2530, 104, - 624, 2530, 0, 1393, 0, 2091, 0, 0, 44, 33, - 586, 2577, 2584, 0, 1038, 1036, 71, 72, 0, 616, - 0, 0, 0, 0, 0, 0, 2217, 24, 0, 0, - 21, 0, 0, 0, 0, 0, 0, 1608, 2486, 2405, - 1764, 1761, 2490, 1661, 0, 5, 9, 1663, 689, 0, - 55, 78, 11, 689, 1892, 684, 2878, 16, 5, 63, - 21, 5, 5, 63, 2224, 756, 757, 664, 5, 13, - 14, 2865, 5, 5, 5, 2228, 5, 5, 2234, 63, - 71, 72, 13, 14, 13, 14, 5, 2881, 168, 115, - 5, 26, 783, 143, 80, 5, 45, 32, 13, 14, - 13, 14, 5, 5, 45, 91, 9, 1081, 5, 5, - 71, 72, 135, 9, 5, 5, 13, 14, 13, 14, - 5, 5, 5, 5, 5, 5, 118, 63, 185, 1925, - 1926, 2378, 167, 3, 4, 5, 176, 120, 4, 9, - 1936, 2724, 98, 9, 1940, 118, 280, 98, 173, 26, - 2306, 2307, 3037, 2309, 2840, 80, 72, 283, 239, 173, - 8, 282, 212, 11, 88, 105, 91, 15, 16, 9, - 30, 19, 20, 21, 929, 1337, 288, 115, 38, 11, - 938, 949, 1156, 1028, 16, 1159, 1160, 2705, 118, 2707, - 3225, 113, 4, 105, 239, 183, 184, 9, 133, 71, - 72, 125, 63, 133, 161, 30, 73, 167, 3, 167, - 235, 793, 268, 38, 46, 207, 1264, 1918, 1919, 1920, - 2449, 235, 932, 368, 404, 1273, 3161, 2448, 116, 2450, - 288, 793, 159, 304, 442, 187, 304, 3268, 288, 34, - 35, 11, 33, 113, 3192, 395, 78, 2360, 164, 157, - 53, 30, 356, 11, 239, 418, 133, 15, 16, 38, - 106, 441, 2500, 251, 252, 267, 3060, 145, 59, 328, - 272, 442, 12, 187, 1155, 959, 46, 17, 920, 106, - 1035, 965, 3317, 462, 2060, 3120, 357, 3122, 46, 29, - 11, 2875, 500, 555, 15, 16, 2825, 1472, 357, 2499, - 2873, 189, 465, 43, 44, 30, 456, 486, 78, 933, - 11, 2884, 1003, 1004, 15, 16, 361, 195, 1086, 3254, - 78, 945, 3257, 3354, 232, 327, 282, 2474, 1019, 500, - 1505, 282, 13, 14, 272, 923, 361, 264, 3160, 981, - 2496, 208, 2498, 213, 348, 305, 306, 307, 348, 3035, - 272, 286, 40, 265, 2401, 436, 2594, 314, 83, 504, - 1522, 3196, 102, 1121, 348, 235, 500, 92, 1944, 172, - 3318, 166, 1130, 168, 500, 299, 361, 365, 366, 500, - 1135, 416, 432, 439, 409, 188, 411, 2905, 318, 114, - 193, 436, 504, 349, 346, 409, 500, 411, 349, 2982, - 1424, 983, 442, 319, 265, 2652, 341, 3292, 368, 286, - 345, 436, 348, 2573, 1750, 2636, 514, 3352, 444, 500, - 500, 983, 436, 563, 495, 1131, 1132, 495, 231, 343, - 416, 1821, 2770, 3255, 11, 458, 284, 2590, 15, 16, - 375, 1083, 1084, 500, 1041, 405, 504, 439, 555, 392, - 393, 436, 2037, 402, 489, 500, 563, 2613, 416, 442, - 185, 402, 3256, 1862, 64, 405, 439, 502, 345, 46, - 347, 498, 72, 198, 614, 502, 1691, 437, 2316, 1108, - 562, 162, 3048, 171, 128, 563, 410, 348, 448, 159, - 2108, 416, 1133, 2109, 1893, 500, 603, 1133, 375, 1081, - 504, 78, 462, 2075, 504, 3037, 462, 614, 3037, 2229, - 445, 462, 2236, 1322, 375, 500, 2101, 433, 451, 603, - 504, 1097, 113, 504, 2095, 385, 486, 464, 40, 104, - 614, 793, 106, 662, 694, 168, 499, 496, 662, 2987, - 500, 2741, 649, 650, 2992, 501, 1570, 503, 603, 486, - 501, 1779, 503, 500, 464, 225, 500, 2748, 1606, 614, - 385, 508, 667, 500, 508, 672, 498, 500, 500, 500, - 502, 500, 500, 1155, 1301, 2941, 486, 380, 2374, 684, - 1836, 500, 1337, 681, 681, 500, 3159, 500, 681, 681, - 500, 681, 670, 1155, 185, 3167, 681, 500, 500, 681, - 3166, 598, 377, 500, 500, 500, 385, 714, 715, 500, - 500, 718, 603, 1949, 681, 500, 500, 500, 500, 500, - 500, 491, 492, 614, 30, 491, 492, 651, 652, 271, - 654, 2860, 681, 277, 681, 235, 2722, 673, 41, 487, - 488, 489, 938, 491, 492, 493, 494, 495, 496, 244, - 462, 491, 492, 614, 1345, 1346, 157, 452, 2996, 171, - 2361, 2362, 2363, 2364, 496, 106, 391, 300, 3252, 394, - 245, 2061, 267, 681, 486, 258, 267, 3015, 253, 491, - 492, 2728, 1706, 681, 1708, 681, 793, 681, 681, 1314, - 11, 1449, 1450, 3266, 681, 251, 252, 1455, 2968, 681, - 2970, 681, 681, 681, 681, 681, 681, 2863, 1463, 681, - 681, 128, 681, 681, 681, 681, 681, 681, 1343, 122, - 128, 983, 128, 116, 831, 500, 3310, 30, 311, 144, - 1905, 232, 1314, 464, 1359, 493, 494, 495, 496, 1787, - 382, 1392, 614, 489, 1395, 26, 1720, 155, 381, 405, - 1401, 32, 1314, 115, 500, 486, 1730, 78, 239, 1733, - 3292, 1343, 1344, 3292, 500, 1407, 1521, 1522, 2544, 1944, - 2546, 1477, 493, 494, 495, 496, 1531, 1359, 279, 194, - 26, 1343, 1344, 1551, 1552, 1553, 32, 503, 1494, 529, - 491, 492, 493, 494, 495, 496, 189, 1359, 2998, 1554, - 221, 239, 501, 2775, 3377, 504, 2338, 464, 0, 365, - 366, 2783, 3358, 920, 1989, 3085, 2401, 2978, 1406, 559, - 452, 928, 929, 1397, 1407, 128, 498, 934, 500, 486, - 937, 938, 1913, 940, 941, 942, 943, 1837, 173, 201, - 221, 923, 2414, 500, 265, 239, 1470, 239, 3394, 956, - 267, 5, 133, 1816, 1451, 1815, 963, 1816, 1817, 1818, - 277, 37, 197, 1837, 272, 555, 464, 1816, 1817, 277, - 347, 2207, 956, 500, 981, 982, 983, 4, 80, 963, - 361, 37, 9, 2263, 265, 343, 42, 133, 486, 91, - 2270, 1649, 2272, 1155, 186, 1002, 1036, 2077, 375, 2079, - 235, 956, 500, 265, 4, 267, 69, 70, 963, 9, - 3218, 3219, 1716, 1020, 491, 492, 493, 494, 495, 496, - 2696, 197, 1573, 361, 1031, 1032, 1033, 763, 1035, 1036, - 246, 1038, 1797, 673, 26, 3135, 1561, 72, 333, 1828, - 32, 59, 4, 99, 3, 2530, 498, 9, 329, 785, - 502, 489, 287, 2181, 1819, 436, 2131, 378, 1823, 1066, - 1689, 1826, 3270, 157, 502, 956, 169, 361, 162, 361, - 13, 14, 963, 23, 24, 370, 1083, 1084, 113, 1561, - 2189, 3210, 1768, 819, 421, 1114, 1115, 1716, 1117, 80, - 1114, 1115, 168, 1117, 405, 956, 499, 378, 436, 1561, - 91, 495, 963, 1108, 197, 286, 1735, 510, 502, 1116, - 405, 1740, 168, 1120, 1121, 423, 13, 14, 439, 500, - 1747, 1748, 1749, 1130, 1131, 1132, 434, 203, 1135, 157, - 4, 454, 498, 2427, 162, 9, 502, 240, 232, 1794, - 286, 133, 436, 219, 436, 2245, 2067, 203, 1155, 167, - 364, 2251, 1314, 229, 2630, 26, 500, 107, 439, 454, - 341, 32, 500, 219, 345, 954, 1171, 169, 499, 162, - 333, 495, 1697, 229, 409, 1726, 411, 167, 502, 510, - 1731, 1343, 1344, 972, 956, 279, 498, 976, 500, 498, - 502, 963, 493, 502, 375, 341, 964, 1359, 433, 345, - 968, 436, 500, 793, 232, 261, 500, 370, 500, 1867, - 1868, 1869, 1870, 1871, 1872, 1697, 500, 1875, 1876, 1877, - 1878, 1879, 1880, 1881, 1882, 1883, 1884, 1844, 500, 375, - 333, 1848, 267, 501, 1851, 1697, 504, 272, 240, 2163, - 2164, 2165, 405, 2728, 501, 6, 1901, 504, 9, 305, - 409, 279, 411, 1449, 1450, 16, 312, 1264, 1462, 1455, - 1464, 1465, 133, 462, 445, 464, 1273, 370, 361, 3369, - 31, 1007, 651, 652, 35, 654, 352, 370, 501, 13, - 14, 504, 1807, 409, 319, 411, 1022, 305, 306, 307, - 1297, 454, 500, 369, 286, 1837, 352, 13, 14, 445, - 335, 2376, 405, 501, 500, 2380, 504, 1314, 3355, 3322, - 3357, 501, 405, 369, 504, 501, 1313, 501, 504, 501, - 504, 1313, 504, 1313, 3337, 1807, 13, 14, 1313, 1854, - 1337, 333, 501, 2627, 500, 504, 1343, 1344, 486, 501, - 501, 1866, 504, 504, 501, 1807, 3393, 504, 500, 341, - 368, 454, 1359, 345, 501, 167, 501, 504, 409, 504, - 411, 454, 2020, 2021, 13, 14, 501, 2512, 370, 504, - 2025, 1896, 1854, 2163, 3387, 2165, 2300, 501, 1903, 3392, - 504, 437, 2497, 375, 1866, 2513, 1393, 405, 501, 500, - 2684, 504, 1854, 983, 2294, 2160, 500, 2162, 433, 1561, - 1407, 13, 14, 405, 1866, 1947, 501, 442, 409, 504, - 411, 2511, 2557, 2513, 1896, 286, 1941, 1424, 500, 437, - 1945, 1903, 501, 1430, 1406, 504, 1951, 218, 501, 1407, - 448, 504, 501, 501, 1896, 504, 504, 3113, 2496, 3115, - 2498, 1903, 1449, 1450, 501, 2072, 501, 504, 1455, 504, - 1457, 501, 454, 445, 504, 1462, 1463, 1464, 1465, 1941, - 502, 501, 1102, 1945, 504, 501, 500, 37, 504, 1951, - 1477, 1478, 483, 1457, 345, 1482, 2134, 1484, 501, 1941, - 1487, 504, 500, 1945, 2405, 1492, 282, 1494, 1495, 1951, - 1497, 166, 501, 501, 1501, 504, 504, 686, 1482, 688, - 1484, 483, 1457, 1487, 375, 502, 40, 501, 1492, 504, - 504, 1495, 500, 1497, 1521, 1522, 501, 1501, 2227, 1526, - 1518, 1528, 2267, 2268, 1531, 1518, 60, 1482, 501, 1484, - 2136, 504, 1487, 13, 14, 1697, 1518, 1492, 13, 14, - 1495, 501, 1497, 1550, 167, 3123, 1501, 1554, 287, 1556, - 59, 1518, 3037, 500, 1561, 1562, 1563, 1564, 1565, 1566, - 1567, 1568, 1569, 1570, 406, 1155, 1457, 1574, 1575, 1518, - 104, 1518, 1579, 483, 445, 218, 1583, 13, 14, 1586, - 1587, 1588, 1589, 1590, 1591, 1592, 1593, 1594, 13, 14, - 1597, 1482, 148, 1484, 13, 14, 1487, 1604, 168, 1606, - 341, 1492, 148, 2527, 1495, 2899, 1497, 2531, 148, 2533, - 1501, 637, 638, 639, 40, 555, 148, 1646, 13, 14, - 1627, 1482, 1646, 1484, 13, 14, 1487, 2742, 167, 13, - 14, 1492, 501, 203, 1495, 500, 1497, 13, 14, 267, - 1501, 40, 1649, 454, 3076, 1807, 596, 1625, 598, 219, - 13, 14, 87, 1660, 1661, 148, 2530, 13, 14, 229, - 466, 2789, 196, 13, 14, 13, 14, 13, 14, 13, - 14, 1867, 1868, 1869, 1870, 1871, 1872, 451, 628, 1875, - 1876, 1877, 1878, 1879, 1880, 1881, 1882, 1883, 1884, 315, - 1697, 500, 1854, 3320, 13, 14, 500, 3324, 148, 1706, - 500, 1708, 500, 653, 1866, 2861, 13, 14, 13, 14, - 1482, 245, 1484, 2238, 500, 1487, 356, 357, 39, 253, - 1492, 356, 357, 1495, 1314, 1497, 255, 256, 500, 1501, - 148, 265, 26, 502, 1896, 365, 366, 2288, 32, 1746, - 148, 1903, 148, 1750, 447, 448, 1753, 1754, 416, 2697, - 500, 3378, 2294, 1343, 1344, 282, 2238, 167, 1398, 501, - 1400, 407, 296, 128, 356, 357, 305, 306, 307, 1359, - 356, 357, 500, 2431, 2432, 500, 2238, 500, 219, 1941, - 1787, 214, 352, 1945, 291, 1783, 500, 1794, 219, 1951, - 155, 219, 500, 3225, 1083, 1084, 40, 2465, 8, 369, - 1807, 11, 289, 230, 500, 15, 16, 3292, 5, 19, - 20, 21, 5, 347, 1821, 500, 500, 500, 500, 143, - 5, 1828, 1829, 5, 2020, 2021, 500, 5, 26, 368, - 1837, 500, 5, 157, 32, 3267, 370, 8, 162, 133, - 11, 2606, 144, 9, 15, 16, 500, 1854, 19, 20, - 21, 1858, 1859, 793, 1861, 389, 463, 504, 375, 1866, - 1867, 1868, 1869, 1870, 1871, 1872, 405, 102, 1875, 1876, - 1877, 1878, 1879, 1880, 1881, 1882, 1883, 1884, 2587, 293, - 504, 2567, 2488, 1890, 1891, 3317, 501, 1894, 212, 1896, - 173, 40, 214, 162, 1901, 162, 1903, 279, 437, 277, - 3045, 59, 267, 230, 500, 416, 2780, 272, 232, 448, - 143, 500, 277, 504, 197, 91, 1923, 416, 59, 59, - 1927, 416, 1929, 2550, 157, 258, 1933, 416, 510, 162, - 416, 462, 148, 98, 1941, 133, 267, 267, 1945, 500, - 1947, 40, 1949, 1927, 1951, 2427, 500, 267, 3, 1933, - 267, 500, 235, 1960, 39, 279, 9, 414, 414, 502, - 498, 500, 96, 498, 288, 2427, 2624, 501, 416, 416, - 920, 1561, 1927, 416, 416, 13, 501, 501, 1933, 212, - 167, 931, 501, 193, 1991, 2025, 2934, 501, 501, 1996, - 1997, 2649, 286, 2914, 504, 2895, 501, 501, 500, 232, - 459, 3157, 219, 274, 287, 2530, 216, 219, 274, 500, - 500, 11, 504, 2020, 2021, 2673, 499, 343, 2025, 458, - 504, 509, 193, 414, 500, 3100, 504, 2792, 2793, 510, - 980, 981, 2039, 167, 272, 2042, 1927, 2044, 500, 416, - 176, 158, 1933, 983, 2825, 216, 279, 341, 2530, 167, - 504, 345, 1692, 2060, 2061, 288, 501, 442, 423, 212, - 2067, 221, 1702, 2070, 1704, 378, 1927, 1707, 2530, 434, - 504, 176, 1933, 1713, 284, 1715, 2238, 2602, 361, 258, - 2087, 375, 283, 304, 304, 214, 2070, 1727, 286, 504, - 501, 504, 1732, 2100, 221, 1045, 1736, 1737, 1738, 1739, - 500, 1741, 1742, 2087, 1054, 2763, 2657, 1697, 432, 221, - 2117, 2118, 267, 284, 288, 2070, 324, 280, 454, 500, - 2602, 500, 1072, 148, 148, 148, 409, 2134, 411, 2136, - 462, 1081, 2087, 1083, 1084, 148, 40, 267, 2145, 282, - 2602, 501, 55, 341, 427, 2627, 501, 345, 483, 498, - 433, 445, 40, 436, 483, 1927, 2163, 2164, 2165, 501, - 501, 1933, 501, 3037, 501, 2627, 282, 3295, 59, 167, - 11, 305, 306, 307, 498, 40, 162, 375, 502, 2070, - 176, 162, 167, 501, 509, 501, 501, 100, 499, 501, - 500, 500, 2232, 501, 2201, 501, 2087, 107, 108, 432, - 2207, 500, 2684, 504, 502, 501, 119, 483, 501, 2070, - 2761, 500, 3136, 416, 500, 1155, 500, 1807, 500, 151, - 500, 40, 2684, 244, 137, 2222, 2087, 459, 141, 59, - 2222, 2238, 2222, 489, 368, 2431, 2432, 2222, 485, 504, - 2898, 283, 283, 439, 504, 238, 59, 445, 59, 2256, - 163, 258, 416, 166, 500, 267, 148, 197, 148, 148, - 416, 501, 416, 274, 1854, 2427, 2791, 40, 181, 416, - 416, 405, 274, 183, 184, 500, 1866, 487, 488, 489, - 343, 491, 492, 493, 494, 495, 496, 2294, 501, 504, - 424, 280, 282, 2300, 462, 40, 148, 501, 272, 499, - 499, 167, 59, 437, 180, 500, 1896, 3016, 500, 2791, - 501, 501, 501, 1903, 448, 501, 487, 488, 489, 162, - 491, 492, 493, 494, 495, 496, 78, 139, 462, 2791, - 193, 2338, 501, 167, 501, 510, 292, 247, 248, 249, - 250, 251, 252, 2825, 501, 255, 256, 283, 500, 500, - 349, 1941, 486, 2895, 2338, 1945, 504, 501, 500, 272, - 176, 1951, 3283, 2825, 501, 2405, 500, 280, 2530, 148, - 3294, 2378, 504, 501, 1314, 500, 171, 427, 501, 501, - 501, 500, 2442, 2338, 40, 40, 501, 2394, 301, 500, - 504, 2398, 2399, 84, 2378, 442, 499, 2437, 2405, 167, - 504, 501, 500, 1343, 1344, 461, 501, 193, 2415, 501, - 504, 2418, 501, 2420, 327, 501, 448, 2899, 3292, 1359, - 2427, 2428, 501, 501, 2431, 2432, 282, 489, 2624, 2436, - 2437, 3086, 59, 3088, 501, 501, 2443, 2899, 3096, 199, - 2602, 501, 2082, 501, 462, 113, 40, 2338, 2488, 500, - 221, 2458, 86, 186, 272, 365, 366, 3125, 2456, 502, - 489, 2468, 272, 2456, 2462, 2627, 502, 502, 502, 502, - 416, 12, 502, 1423, 2456, 3102, 17, 1427, 40, 416, - 502, 2488, 502, 502, 40, 502, 502, 502, 29, 2456, - 555, 3101, 3098, 502, 3104, 499, 502, 502, 60, 502, - 267, 502, 43, 44, 499, 502, 3, 2456, 5, 2456, - 502, 502, 3037, 105, 502, 502, 40, 501, 3227, 502, - 2527, 500, 2684, 2530, 2531, 502, 2533, 502, 462, 502, - 502, 502, 3236, 502, 502, 502, 502, 2544, 282, 2546, - 8, 500, 104, 9, 342, 500, 500, 15, 16, 326, - 501, 19, 20, 21, 504, 3037, 504, 59, 501, 504, - 193, 102, 186, 7, 8, 2563, 89, 2763, 12, 2566, - 480, 481, 447, 17, 2566, 3037, 2566, 21, 501, 23, - 24, 2566, 72, 500, 28, 29, 30, 335, 78, 33, - 3217, 2589, 36, 37, 2592, 2602, 40, 40, 113, 43, - 44, 91, 148, 501, 120, 3211, 148, 40, 2248, 2649, - 107, 108, 40, 40, 501, 357, 357, 2624, 501, 504, - 2627, 1561, 500, 113, 301, 115, 500, 71, 72, 2791, - 500, 500, 272, 442, 196, 243, 185, 2644, 2645, 442, - 427, 500, 2649, 72, 285, 2652, 72, 78, 2238, 9, - 94, 501, 167, 501, 2661, 500, 359, 101, 102, 103, - 104, 105, 8, 2825, 1614, 11, 2673, 499, 2652, 15, - 16, 2678, 2679, 501, 555, 499, 2683, 2684, 2718, 59, - 91, 2688, 129, 245, 2691, 2692, 183, 184, 489, 2696, - 2697, 253, 3189, 2700, 265, 282, 427, 2704, 40, 285, - 46, 500, 2898, 265, 2711, 143, 285, 53, 500, 447, - 501, 201, 501, 3264, 199, 282, 501, 501, 282, 157, - 2704, 501, 375, 118, 162, 439, 356, 25, 793, 36, - 1714, 2158, 78, 2527, 296, 2514, 3127, 2899, 2466, 681, - 3267, 3221, 3368, 1828, 2785, 3347, 3242, 3009, 2755, 2704, - 247, 248, 249, 250, 251, 252, 2763, 1697, 255, 256, - 3305, 3296, 3340, 3062, 2151, 2164, 3294, 3292, 1142, 2578, - 3303, 2506, 2148, 3291, 212, 2782, 2533, 267, 1392, 1343, - 295, 1311, 272, 0, 2791, 347, 2221, 2300, 2895, 2134, - 305, 306, 307, 955, 232, 141, 2559, 1685, 1102, 2359, - 2100, 18, 1649, 3276, 21, 955, 3198, 2117, 370, 1123, - 3292, 3090, 1684, 2704, 21, 2938, 284, 2087, 2825, 36, - 1939, 1124, 603, 933, 614, 2337, 172, 389, 45, 319, - 3292, 1126, 2652, 2338, 2386, 3030, 3202, 2427, 1361, 1837, - 931, 279, 188, 2704, 931, 335, 3201, 193, 931, 931, - 288, 2491, 2492, 368, 71, 72, 73, 1998, 3333, 2399, - 2867, 2101, 1961, 2043, 2000, 2040, 2711, 1807, 365, 366, - 1478, 2412, 2456, 3187, 2914, 3037, 597, 94, 2876, 2519, - 1947, 1407, 1427, 1626, 2467, 231, 14, 1837, 2895, 1625, - 405, 2898, 2899, 1843, 2534, 2535, 2536, 2537, 2538, 2539, - 2540, 2541, 2542, 2543, 94, -1, -1, 2914, -1, -1, - -1, -1, 793, -1, 1854, -1, -1, -1, 983, -1, - -1, -1, 437, -1, -1, -1, 1866, 2934, -1, -1, - -1, -1, 2704, 448, 2941, -1, -1, -1, 284, -1, - 2530, -1, -1, 433, -1, 2952, 2953, 462, -1, 2956, - -1, 2958, 442, -1, -1, -1, 1896, -1, -1, -1, - -1, -1, -1, 1903, -1, -1, -1, -1, -1, -1, - -1, 486, -1, -1, -1, -1, 2983, -1, -1, 40, - -1, -1, -1, 480, 481, 500, -1, -1, 529, -1, - -1, -1, -1, -1, 432, -1, -1, 1947, -1, -1, - 3007, 1941, -1, -1, -1, 1945, -1, -1, -1, -1, - -1, 1951, 2602, -1, -1, -1, -1, 558, 559, 487, - 488, 489, -1, 491, 492, 493, 494, 495, 496, -1, - 3037, -1, -1, -1, 380, -1, -1, 2627, 147, -1, - 101, -1, 103, -1, 105, -1, -1, -1, -1, -1, - -1, -1, 3050, 3051, -1, -1, -1, -1, 167, -1, - 498, -1, -1, 501, 502, 3072, -1, -1, -1, 3076, - -1, 515, -1, -1, -1, 519, -1, -1, 522, 3086, - -1, 3088, -1, 3090, -1, 529, -1, 3094, -1, 3096, - 1155, 3098, 3076, -1, 2684, -1, -1, -1, -1, -1, - -1, 3108, 983, -1, -1, -1, 3113, -1, 3115, -1, - -1, -1, -1, 557, 558, 559, 3123, -1, -1, -1, - -1, 3076, 2762, -1, -1, -1, 3124, -1, 3126, 3136, - 3292, -1, 673, -1, 3141, -1, -1, -1, 484, -1, - -1, -1, -1, -1, -1, 491, 492, 493, 494, 495, - 496, -1, 596, 597, 598, -1, -1, -1, -1, 603, - -1, 605, -1, -1, -1, -1, -1, 3165, 40, -1, - 614, -1, -1, -1, 618, 619, -1, -1, -1, 623, - 624, 3188, 147, 3181, -1, 3076, -1, 3185, 60, -1, - -1, 3198, -1, 637, 638, 639, 305, 306, 307, -1, - -1, 2791, 167, -1, 3211, -1, -1, 651, 652, -1, - 654, -1, -1, -1, -1, -1, -1, -1, 3225, -1, - 2170, -1, -1, 667, -1, -1, -1, -1, -1, 673, - -1, -1, 104, -1, -1, 2825, 3243, -1, -1, -1, - 684, 3225, -1, -1, -1, -1, -1, -1, -1, 1314, - -1, -1, -1, -1, 698, -1, -1, -1, -1, 368, - 3267, 0, -1, -1, -1, 2215, -1, -1, -1, -1, - 3225, -1, -1, -1, 1155, -1, 3283, -1, 1343, 1344, - -1, -1, 2232, 3267, -1, 3292, -1, 3294, 732, 733, - 734, 735, -1, -1, 1359, -1, 405, 514, 2238, 171, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 2899, - 3317, 3309, 3267, 37, -1, -1, -1, -1, 42, -1, - -1, -1, -1, -1, 196, 3332, 3333, -1, 437, -1, - -1, -1, -1, 3317, 3225, -1, -1, 3335, -1, 448, - 305, 306, 307, -1, 2294, 562, 563, -1, -1, -1, - -1, 3358, 2302, 462, 2304, -1, -1, -1, 2308, 98, - 2310, -1, 3317, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 245, -1, 99, 3267, 486, 3018, -1, - -1, 253, -1, -1, -1, -1, 603, 3394, -1, -1, - -1, 500, -1, 265, -1, -1, -1, 614, 3038, 3039, - -1, -1, -1, 368, 143, -1, -1, -1, -1, 950, - -1, -1, -1, -1, -1, -1, -1, -1, 157, 3059, - -1, -1, -1, 162, 296, -1, 3317, -1, 167, -1, - -1, -1, 973, 1314, -1, -1, -1, 176, -1, -1, - 405, 180, -1, -1, 168, 662, -1, 3037, -1, -1, - -1, -1, 176, -1, 515, -1, -1, -1, 519, -1, - -1, 522, 1343, 1344, 681, -1, -1, -1, -1, -1, - -1, -1, 437, 212, -1, 347, -1, -1, 1359, 203, - -1, -1, -1, 448, -1, -1, -1, 2427, 932, 933, - -1, 935, -1, 232, 938, 219, 1561, 462, 370, 1040, - 944, 945, 37, -1, -1, 229, 950, 42, -1, -1, - -1, -1, 956, -1, -1, 8, -1, 389, 11, 963, - -1, 486, 15, 16, -1, -1, 19, 20, 21, 973, - -1, -1, -1, -1, -1, 500, 980, 261, -1, -1, - 279, -1, -1, 282, 605, -1, 990, -1, -1, 288, - -1, -1, -1, 46, -1, -1, -1, -1, 282, -1, - 53, 1102, 1103, -1, 99, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 2530, 305, -1, -1, -1, 78, -1, 326, 312, -1, - 2530, 8, -1, -1, 11, -1, 1040, -1, 15, 16, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 2559, - 349, -1, -1, -1, -1, -1, -1, -1, 8, -1, - -1, 11, -1, -1, -1, 15, 16, -1, 352, 46, - -1, -1, 1697, 168, -1, -1, 53, 698, -1, 8, - -1, 1085, 11, 1087, -1, 369, 15, 16, 3288, -1, - 19, 20, 21, -1, -1, -1, 46, -1, 1102, 1103, - -1, 78, 2602, 53, 1108, -1, 405, 36, 203, -1, - -1, 732, 733, 734, 735, -1, -1, -1, -1, 172, - 1561, -1, -1, -1, 219, -1, 1130, 2627, 78, -1, - -1, -1, -1, 432, 229, 188, -1, 1141, 1142, -1, - 193, -1, 3292, 442, 2654, -1, 923, -1, -1, -1, - 1154, -1, -1, 437, -1, -1, -1, -1, 442, -1, - 459, -1, 461, 462, 141, -1, 261, 1171, -1, -1, - -1, 1175, -1, -1, -1, 1179, -1, -1, 231, 956, - -1, -1, 1807, -1, 2684, -1, 963, 282, -1, -1, - -1, 141, -1, -1, -1, 172, -1, -1, 2708, 498, - -1, -1, 501, 502, 503, -1, -1, -1, -1, -1, - 305, 188, -1, -1, -1, -1, 193, 312, -1, -1, - -1, 998, 172, -1, -1, -1, -1, -1, -1, 1854, - -1, 284, 1009, -1, -1, -1, -1, -1, 188, -1, - -1, 1866, -1, 193, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 231, -1, 1697, 352, -1, 1036, + 7, 626, 711, 560, 501, 0, 69, 0, 719, 796, + 0, 943, 0, 0, 21, 1106, 0, 1183, 0, 18, + 558, 0, 1170, 71, 72, 566, 1172, 675, 923, 723, + 802, 1460, 0, 558, 18, 0, 1145, 985, 1646, 45, + 2105, 1508, 728, 36, 834, 73, 21, 1139, 1607, 923, + 827, 73, 7, 830, 652, 2074, 934, 2076, 1147, 1347, + 697, 1533, 1352, 1749, 71, 72, 21, 934, 934, 1150, + 1300, 37, 2160, 711, 2095, 713, 558, 715, 12, 934, + 2207, 2208, 2209, 17, 565, 2567, 1175, 1560, 1561, 622, + 561, 566, 0, 2533, 627, 29, 71, 72, 2565, 2040, + 1073, 94, 0, 1396, 2094, 1078, 2533, 1039, 44, 43, + 44, 0, 0, 2156, 0, 0, 71, 72, 0, 0, + 0, 0, 619, 1041, 33, 0, 0, 0, 0, 0, + 21, 0, 0, 2408, 2587, 2580, 0, 589, 104, 759, + 760, 0, 7, 5, 0, 0, 24, 2220, 1611, 1764, + 11, 13, 14, 2489, 0, 16, 21, 2493, 1767, 11, + 9, 55, 13, 14, 16, 5, 786, 1664, 102, 1895, + 667, 692, 5, 23, 24, 5, 1666, 5, 2881, 0, + 71, 72, 692, 5, 687, 46, 5, 13, 14, 72, + 5, 13, 14, 2227, 2237, 5, 63, 5, 13, 14, + 5, 9, 78, 5, 2231, 5, 71, 72, 13, 14, + 1084, 5, 115, 13, 14, 9, 4, 78, 45, 168, + 5, 9, 5, 30, 167, 45, 118, 5, 63, 5, + 143, 38, 5, 5, 5, 5, 5, 30, 63, 5, + 30, 952, 2381, 143, 157, 38, 63, 3040, 38, 162, + 1928, 1929, 98, 9, 72, 118, 2727, 107, 186, 1031, + 176, 1939, 3, 4, 5, 1943, 2309, 2310, 9, 2312, + 284, 796, 88, 2773, 105, 135, 63, 98, 120, 8, + 128, 164, 11, 2843, 932, 1159, 15, 16, 1162, 1163, + 19, 20, 21, 289, 240, 113, 1340, 11, 240, 766, + 213, 15, 16, 3, 281, 370, 102, 155, 240, 125, + 935, 240, 283, 213, 796, 113, 208, 2868, 73, 1267, + 233, 788, 188, 168, 113, 2452, 179, 80, 1276, 1921, + 1922, 1923, 46, 2884, 34, 35, 3164, 105, 91, 118, + 3271, 240, 306, 167, 2363, 11, 128, 306, 269, 15, + 16, 188, 289, 40, 133, 822, 358, 2971, 161, 2973, + 2451, 3228, 2453, 40, 78, 41, 11, 280, 159, 115, + 128, 1158, 113, 2503, 80, 421, 289, 2063, 1089, 1865, + 176, 406, 923, 465, 1475, 91, 1006, 1007, 2828, 289, + 1038, 3195, 407, 4, 2876, 359, 157, 186, 9, 3221, + 3222, 46, 1022, 936, 11, 2887, 3362, 489, 15, 16, + 1896, 2878, 2477, 245, 2502, 948, 2708, 1508, 2710, 444, + 268, 363, 468, 13, 14, 273, 116, 3358, 106, 3257, + 278, 363, 3260, 78, 363, 226, 268, 283, 321, 46, + 64, 266, 507, 984, 2778, 926, 122, 3403, 72, 106, + 268, 3273, 2786, 3320, 209, 273, 2499, 302, 2501, 379, + 3163, 986, 283, 2404, 363, 194, 166, 2597, 168, 106, + 266, 78, 233, 214, 3088, 273, 419, 330, 3038, 268, + 1947, 1525, 348, 350, 171, 301, 268, 11, 217, 503, + 1138, 15, 16, 439, 171, 236, 278, 439, 266, 2999, + 190, 503, 3295, 321, 986, 351, 359, 439, 345, 1427, + 439, 507, 3063, 316, 2985, 350, 2655, 3321, 3018, 337, + 278, 442, 435, 1753, 517, 350, 503, 273, 3356, 445, + 351, 1824, 503, 350, 498, 435, 1134, 1135, 383, 498, + 439, 320, 2576, 1010, 447, 1086, 1087, 1044, 2639, 492, + 442, 558, 377, 436, 503, 3258, 285, 503, 1025, 566, + 566, 503, 505, 350, 394, 395, 2593, 566, 2040, 617, + 507, 503, 162, 2616, 503, 503, 407, 404, 426, 442, + 387, 565, 4, 503, 404, 1694, 106, 9, 501, 437, + 116, 504, 505, 2319, 387, 419, 412, 387, 532, 606, + 3040, 461, 1084, 445, 503, 2078, 3051, 503, 1111, 503, + 617, 617, 236, 3040, 2111, 1136, 2908, 2232, 436, 465, + 416, 454, 2112, 69, 70, 501, 1136, 445, 562, 505, + 2239, 1325, 2104, 1158, 697, 2098, 1782, 665, 499, 5, + 507, 503, 617, 665, 465, 652, 653, 499, 1100, 511, + 397, 606, 503, 502, 2990, 1573, 2744, 467, 504, 2995, + 506, 1609, 617, 503, 190, 467, 419, 1304, 675, 2944, + 503, 511, 507, 1839, 673, 503, 1158, 503, 2751, 489, + 3162, 503, 501, 504, 503, 506, 505, 489, 503, 684, + 507, 684, 1340, 503, 684, 503, 684, 684, 503, 2377, + 684, 503, 684, 503, 670, 684, 494, 495, 3259, 503, + 717, 718, 459, 419, 721, 606, 684, 3170, 503, 684, + 503, 687, 1952, 601, 3169, 503, 617, 503, 1348, 1349, + 503, 503, 503, 503, 503, 2725, 2863, 503, 494, 495, + 676, 606, 676, 445, 5, 654, 655, 159, 657, 599, + 157, 601, 617, 494, 495, 455, 252, 253, 59, 30, + 30, 490, 491, 492, 115, 494, 495, 496, 497, 498, + 499, 2064, 2364, 2365, 2366, 2367, 684, 654, 655, 1317, + 657, 631, 496, 497, 498, 499, 684, 3269, 3255, 796, + 2731, 1709, 1317, 1711, 445, 684, 684, 26, 684, 684, + 11, 503, 684, 684, 684, 684, 656, 272, 1346, 684, + 684, 684, 684, 684, 268, 684, 684, 1908, 1466, 273, + 684, 1346, 1347, 2866, 1362, 684, 233, 834, 684, 684, + 496, 497, 498, 499, 3123, 1317, 3125, 1362, 1395, 104, + 0, 1398, 1790, 1554, 1555, 1556, 3313, 1404, 33, 1723, + 506, 202, 503, 265, 465, 3295, 1947, 128, 128, 1733, + 259, 2547, 1736, 2549, 1346, 1347, 167, 78, 3295, 1410, + 455, 367, 368, 280, 59, 329, 1524, 1525, 489, 222, + 1362, 53, 1480, 494, 495, 507, 1534, 494, 495, 496, + 497, 498, 499, 184, 185, 492, 26, 503, 240, 1497, + 3382, 1992, 32, 411, 133, 413, 167, 144, 505, 1557, + 3199, 959, 2341, 3001, 313, 266, 923, 268, 966, 384, + 962, 128, 345, 266, 931, 932, 968, 169, 1409, 1400, + 937, 145, 2404, 940, 941, 1410, 943, 944, 945, 946, + 1473, 1819, 926, 1916, 2417, 1840, 173, 411, 155, 413, + 26, 1818, 959, 1819, 1820, 1821, 32, 1454, 195, 966, + 2981, 252, 253, 4, 1819, 1820, 1840, 198, 9, 96, + 494, 495, 496, 497, 498, 499, 2266, 984, 985, 986, + 2210, 246, 196, 2273, 959, 2275, 498, 157, 2080, 254, + 2082, 966, 162, 505, 492, 498, 157, 349, 1005, 241, + 172, 162, 505, 133, 959, 503, 307, 308, 309, 236, + 1719, 966, 198, 2699, 26, 173, 1023, 189, 167, 1576, + 32, 363, 194, 1800, 240, 377, 1564, 1034, 1035, 1036, + 335, 1038, 1039, 1039, 1041, 13, 14, 380, 2184, 1564, + 167, 1831, 467, 2134, 80, 1822, 307, 308, 309, 1826, + 3138, 4, 1829, 187, 1692, 91, 9, 133, 287, 80, + 232, 2533, 1069, 233, 489, 3, 273, 372, 959, 370, + 91, 278, 233, 923, 247, 966, 367, 368, 236, 1086, + 1087, 1719, 1564, 4, 934, 1771, 3213, 501, 9, 1117, + 1118, 505, 1120, 335, 959, 1117, 1118, 439, 1120, 442, + 1738, 966, 407, 2192, 501, 1743, 407, 467, 505, 370, + 280, 502, 1119, 1750, 1751, 1752, 1123, 1124, 347, 280, + 349, 133, 513, 411, 502, 413, 1133, 1134, 1135, 489, + 372, 1138, 363, 983, 984, 513, 363, 37, 2070, 440, + 501, 372, 2430, 503, 505, 1111, 407, 363, 377, 1797, + 451, 1158, 457, 13, 14, 335, 957, 287, 307, 308, + 309, 503, 1700, 13, 14, 407, 2633, 2248, 640, 641, + 642, 1105, 1729, 2254, 975, 1700, 407, 1734, 979, 440, + 307, 308, 309, 26, 411, 501, 413, 503, 8, 32, + 451, 11, 372, 169, 222, 15, 16, 457, 1048, 19, + 20, 21, 503, 1465, 465, 1467, 1468, 1057, 1174, 467, + 382, 287, 439, 343, 424, 457, 36, 347, 1700, 426, + 501, 370, 503, 439, 505, 1075, 457, 407, 489, 967, + 437, 489, 504, 971, 1084, 507, 1086, 1087, 266, 13, + 14, 6, 503, 370, 9, 503, 366, 377, 2166, 2167, + 2168, 16, 465, 411, 467, 413, 1904, 1847, 407, 2731, + 1267, 1851, 13, 14, 1854, 241, 31, 343, 168, 1276, + 35, 347, 1810, 13, 14, 287, 504, 457, 503, 507, + 407, 439, 411, 407, 413, 1810, 3374, 503, 2379, 167, + 133, 440, 2383, 1300, 3359, 504, 3361, 503, 507, 1840, + 427, 377, 451, 331, 204, 504, 3325, 503, 507, 504, + 1317, 26, 507, 440, 2166, 162, 2168, 32, 448, 1857, + 220, 1316, 3341, 2499, 451, 2501, 1316, 503, 1810, 1316, + 230, 1869, 1857, 1340, 689, 347, 691, 3402, 465, 1346, + 1347, 503, 2630, 496, 1869, 13, 14, 504, 1316, 503, + 507, 503, 380, 504, 503, 1362, 507, 504, 503, 335, + 507, 1899, 489, 26, 503, 377, 503, 2515, 1906, 32, + 2516, 503, 448, 489, 1899, 1857, 503, 3396, 13, 14, + 2028, 1906, 3401, 358, 359, 2303, 2163, 1869, 2165, 1396, + 504, 2500, 504, 507, 167, 507, 372, 217, 504, 2687, + 503, 507, 2297, 1410, 504, 504, 1944, 507, 507, 1950, + 1948, 1410, 2560, 486, 442, 219, 1954, 1899, 133, 1944, + 1427, 13, 14, 1948, 1906, 1409, 1433, 13, 14, 1954, + 3116, 407, 3118, 2514, 505, 2516, 448, 1485, 2075, 1487, + 504, 283, 1490, 507, 287, 1452, 1453, 1495, 13, 14, + 1498, 1458, 1500, 1460, 354, 486, 1504, 505, 1465, 1466, + 1467, 1468, 1944, 13, 14, 285, 1948, 1401, 166, 1403, + 133, 371, 1954, 1480, 1481, 507, 2408, 503, 1485, 504, + 1487, 457, 507, 1490, 504, 504, 504, 507, 1495, 507, + 1497, 1498, 167, 1500, 13, 14, 504, 1504, 2270, 2271, + 343, 13, 14, 504, 347, 1460, 507, 288, 504, 503, + 1485, 507, 1487, 504, 59, 1490, 507, 1524, 1525, 408, + 1495, 486, 1529, 1498, 1531, 1500, 219, 1534, 1521, 1504, + 1485, 148, 1487, 1521, 377, 1490, 2230, 1521, 343, 1521, + 1495, 2139, 1521, 1498, 504, 1500, 1553, 507, 504, 1504, + 1557, 507, 1559, 13, 14, 148, 1521, 1564, 1565, 1566, + 1567, 1568, 1569, 1570, 1571, 1572, 1573, 3126, 3040, 1460, + 1577, 1578, 287, 148, 504, 1582, 1426, 507, 504, 1586, + 1430, 507, 1589, 1590, 1591, 1592, 1593, 1594, 1595, 1596, + 1597, 13, 14, 1600, 1485, 1460, 1487, 148, 504, 1490, + 1607, 507, 1609, 504, 1495, 448, 507, 1498, 504, 1500, + 40, 507, 2530, 1504, 2902, 503, 2534, 504, 2536, 268, + 1485, 1649, 1487, 1630, 287, 1490, 40, 1649, 343, 1628, + 1495, 504, 347, 1498, 507, 1500, 2745, 504, 457, 1504, + 507, 504, 13, 14, 507, 1652, 2792, 87, 504, 504, + 3079, 507, 507, 148, 143, 504, 1663, 1664, 507, 2533, + 13, 14, 377, 469, 173, 13, 14, 454, 157, 503, + 490, 491, 492, 162, 494, 495, 496, 497, 498, 499, + 343, 13, 14, 317, 347, 13, 14, 503, 8, 198, + 148, 11, 503, 1700, 503, 15, 16, 39, 2864, 19, + 20, 21, 1709, 2241, 1711, 13, 14, 26, 503, 13, + 14, 13, 14, 32, 377, 3323, 2241, 72, 503, 3327, + 13, 14, 148, 78, 213, 30, 505, 236, 40, 358, + 359, 256, 257, 448, 2291, 148, 91, 358, 359, 367, + 368, 148, 1749, 283, 233, 419, 1753, 503, 60, 1756, + 1757, 167, 2700, 358, 359, 503, 2297, 2468, 113, 2241, + 115, 1695, 450, 451, 3336, 3337, 503, 1617, 113, 3368, + 3369, 1705, 504, 1707, 503, 3383, 1710, 37, 83, 288, + 1086, 1087, 1716, 1790, 1718, 448, 409, 92, 215, 220, + 1797, 280, 104, 1786, 293, 503, 1730, 220, 503, 3228, + 289, 1735, 290, 1810, 40, 1739, 1740, 1741, 1742, 114, + 1744, 1745, 220, 503, 133, 5, 231, 1824, 5, 503, + 503, 503, 167, 3295, 1831, 1832, 503, 5, 5, 5, + 503, 503, 2609, 1840, 5, 144, 9, 503, 295, 466, + 507, 3270, 143, 102, 504, 507, 40, 202, 215, 377, + 1857, 162, 280, 278, 1861, 1862, 157, 1864, 59, 162, + 503, 162, 1869, 1870, 1871, 1872, 1873, 1874, 1875, 231, + 419, 1878, 1879, 1880, 1881, 1882, 1883, 1884, 1885, 1886, + 1887, 186, 1930, 503, 2570, 197, 1893, 1894, 1936, 91, + 1897, 3320, 1899, 2491, 199, 507, 2590, 1904, 419, 1906, + 3048, 59, 411, 59, 413, 2430, 419, 259, 168, 2783, + 513, 419, 213, 268, 419, 148, 2553, 465, 273, 1926, + 503, 268, 268, 1930, 98, 1932, 40, 436, 268, 1936, + 439, 268, 233, 503, 246, 3, 503, 1944, 9, 39, + 417, 1948, 254, 1950, 204, 1952, 435, 1954, 2430, 417, + 419, 505, 297, 419, 266, 1930, 1963, 501, 501, 13, + 220, 1936, 307, 308, 309, 285, 321, 419, 287, 419, + 230, 504, 504, 504, 167, 1930, 462, 504, 504, 280, + 503, 1936, 337, 507, 558, 2917, 298, 1994, 289, 2937, + 1840, 220, 1999, 2000, 3160, 2533, 1846, 504, 504, 220, + 275, 275, 503, 2898, 507, 503, 461, 11, 2533, 345, + 502, 507, 3103, 513, 507, 512, 2023, 2024, 2795, 2796, + 417, 2028, 2028, 503, 343, 370, 273, 503, 347, 419, + 176, 158, 3, 167, 5, 2042, 507, 349, 2045, 1930, + 2047, 2828, 504, 445, 213, 1936, 507, 259, 306, 380, + 222, 2533, 284, 306, 507, 176, 2063, 2064, 377, 507, + 372, 8, 407, 2070, 11, 1930, 2073, 2605, 15, 16, + 215, 1936, 19, 20, 21, 941, 173, 268, 222, 391, + 2605, 436, 503, 2090, 504, 222, 289, 326, 393, 36, + 445, 396, 281, 457, 354, 440, 2103, 503, 2073, 503, + 1950, 198, 148, 2660, 148, 2630, 451, 148, 465, 148, + 40, 371, 504, 2120, 2121, 2090, 268, 283, 2073, 504, + 465, 501, 504, 2605, 283, 486, 486, 504, 504, 448, + 2137, 504, 2139, 40, 435, 2090, 107, 108, 59, 236, + 558, 2148, 8, 167, 489, 11, 11, 40, 2630, 15, + 16, 2085, 3298, 19, 20, 21, 162, 176, 503, 2166, + 2167, 2168, 2687, 504, 162, 504, 3040, 505, 503, 147, + 490, 491, 492, 503, 494, 495, 496, 497, 498, 499, + 504, 504, 2073, 504, 167, 503, 502, 512, 37, 167, + 507, 288, 504, 42, 504, 486, 503, 2204, 504, 2090, + 501, 419, 151, 2210, 505, 2687, 504, 2764, 2073, 503, + 7, 8, 245, 184, 185, 12, 503, 503, 503, 462, + 17, 3139, 796, 40, 21, 2090, 23, 24, 59, 2235, + 2225, 28, 29, 30, 2241, 2225, 33, 507, 2225, 36, + 37, 492, 488, 40, 284, 442, 43, 44, 284, 239, + 99, 507, 2259, 59, 59, 419, 2794, 2225, 1124, 259, + 268, 503, 148, 198, 148, 148, 363, 1133, 419, 2794, + 217, 419, 419, 275, 71, 72, 419, 248, 249, 250, + 251, 252, 253, 275, 504, 256, 257, 503, 40, 345, + 2297, 504, 507, 281, 283, 465, 2303, 94, 40, 504, + 148, 273, 502, 2828, 101, 102, 103, 104, 105, 504, + 502, 941, 2794, 167, 411, 503, 413, 2251, 503, 168, + 504, 504, 504, 2173, 59, 3019, 181, 162, 194, 307, + 308, 309, 78, 430, 2341, 139, 504, 194, 285, 436, + 167, 504, 439, 503, 294, 507, 2828, 504, 513, 503, + 351, 217, 284, 503, 3286, 204, 176, 2898, 148, 171, + 40, 430, 504, 504, 40, 504, 504, 507, 2218, 504, + 503, 220, 504, 503, 2381, 504, 507, 2902, 796, 3297, + 503, 230, 2445, 84, 445, 2235, 2341, 502, 507, 167, + 2397, 503, 370, 504, 2401, 2402, 367, 368, 464, 504, + 194, 2408, 2408, 504, 507, 504, 504, 283, 504, 504, + 492, 2418, 986, 262, 2421, 59, 2423, 3128, 504, 285, + 2902, 3295, 451, 2430, 2431, 504, 504, 2434, 2435, 407, + 504, 465, 2439, 2440, 2440, 200, 113, 40, 503, 2446, + 222, 3089, 187, 3091, 86, 273, 505, 2297, 273, 492, + 2341, 419, 505, 505, 2461, 2305, 505, 2307, 307, 419, + 505, 2311, 440, 2313, 2471, 314, 2459, 505, 3105, 40, + 505, 2459, 2465, 451, 505, 2459, 2341, 2459, 505, 3104, + 2459, 502, 3107, 505, 2491, 2491, 504, 465, 558, 505, + 505, 505, 505, 505, 2459, 505, 505, 268, 505, 502, + 505, 105, 3040, 3101, 465, 354, 505, 40, 505, 505, + 505, 489, 483, 484, 505, 3040, 2381, 283, 505, 505, + 505, 505, 371, 2530, 505, 503, 2533, 2534, 505, 2536, + 3239, 505, 505, 503, 503, 9, 3230, 344, 503, 503, + 2547, 507, 2549, 490, 491, 492, 504, 494, 495, 496, + 497, 498, 499, 328, 507, 59, 504, 194, 3040, 187, + 2494, 2495, 507, 450, 89, 504, 503, 337, 986, 40, + 148, 504, 120, 2566, 2569, 40, 148, 504, 359, 2569, + 359, 40, 2569, 3220, 1158, 504, 1452, 1453, 2522, 503, + 503, 440, 1458, 40, 147, 507, 303, 503, 2605, 2592, + 445, 2569, 2595, 2537, 2538, 2539, 2540, 2541, 2542, 2543, + 2544, 2545, 2546, 503, 167, 273, 3214, 244, 186, 445, + 2627, 430, 503, 2630, 490, 491, 492, 72, 494, 495, + 496, 497, 498, 499, 286, 72, 78, 9, 361, 503, + 2647, 2648, 40, 504, 504, 2652, 2652, 59, 2655, 129, + 502, 504, 502, 91, 492, 266, 430, 2664, 283, 2707, + 40, 503, 60, 286, 286, 504, 503, 450, 504, 2676, + 504, 504, 200, 283, 2681, 2682, 283, 377, 118, 2686, + 2687, 504, 442, 2533, 2691, 358, 144, 2694, 2695, 25, + 36, 358, 2699, 2700, 291, 3192, 2703, 3130, 2161, 3270, + 2707, 1717, 684, 2517, 2469, 3224, 104, 2714, 3373, 1831, + 3267, 2788, 2562, 3245, 3012, 2721, 3351, 2530, 3299, 3308, + 3344, 518, 3065, 2154, 2167, 522, 796, 2581, 525, 3297, + 3306, 2151, 2707, 1145, 2509, 532, 3294, 2536, 1395, 1346, + 1158, 1314, 2224, 1317, 2303, 2898, 1105, 2137, 958, 958, + 2562, 2758, 2707, 2103, 307, 308, 309, 3295, 1127, 2766, + 1688, 1126, 2362, 560, 561, 562, 3279, 2120, 1652, 3201, + 3295, 3093, 1346, 1347, 40, 1687, 21, 2090, 2785, 1129, + 2941, 936, 2340, 1942, 606, 2655, 1652, 2794, 1362, 2341, + 2655, 2389, 3033, 617, 60, 1364, 1840, 3205, 934, 197, + 3204, 2001, 599, 600, 601, 2104, 934, 2657, 934, 606, + 934, 608, 2046, 3295, 2402, 2003, 2707, 370, 1964, 2043, + 617, 2828, 1452, 1453, 621, 622, 1481, 2714, 1458, 626, + 627, 2765, 2415, 2459, 3190, 1950, 600, 1628, 104, 105, + 1430, 2470, 2707, 640, 641, 642, 1410, 113, 246, 1629, + 14, 94, -1, -1, 407, -1, 254, 654, 655, -1, + 657, 2711, -1, 2870, -1, -1, -1, -1, 266, -1, + -1, -1, -1, 670, -1, -1, -1, -1, -1, 676, + -1, -1, -1, -1, -1, -1, 2879, 440, -1, -1, + 687, 2898, -1, -1, 2901, 2902, -1, -1, 451, 1317, + 298, -1, -1, -1, 701, 171, -1, -1, -1, -1, + 2917, 2917, 465, -1, -1, -1, 986, -1, -1, -1, + -1, -1, -1, -1, -1, 558, -1, -1, 1346, 1347, + 2937, 197, -1, -1, -1, -1, 489, 2944, 735, 736, + 737, 738, -1, -1, 1362, -1, -1, -1, 2955, 2956, + 503, 349, 2959, -1, 2961, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 40, -1, -1, -1, + -1, -1, -1, -1, 372, -1, -1, -1, -1, 2986, + 246, -1, -1, -1, 8, 558, 60, 11, 254, -1, + 1564, 15, 16, 391, -1, 19, 20, 21, -1, -1, + 266, -1, 268, 3010, 1870, 1871, 1872, 1873, 1874, 1875, + -1, -1, 1878, 1879, 1880, 1881, 1882, 1883, 1884, 1885, + 1886, 1887, 46, -1, -1, 2875, -1, -1, -1, 53, + 104, -1, 298, 3040, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 2898, -1, + -1, -1, -1, -1, 78, -1, -1, -1, -1, -1, + 3053, 3054, -1, 8, -1, -1, 11, -1, 3075, -1, + 15, 16, 3079, -1, 19, 20, 21, -1, -1, -1, + -1, -1, 3089, 349, 3091, -1, 3093, 3021, 1158, -1, + 3097, 36, 3099, -1, 3101, -1, -1, 171, -1, -1, + -1, -1, -1, -1, 3111, 8, 372, 3041, 3042, 3116, + -1, 3118, 15, 16, -1, -1, 19, 20, 21, 3126, + -1, -1, 2972, 197, 3079, 391, 1700, 393, 3062, -1, + 396, -1, 3139, -1, 3127, -1, 3129, 3144, 935, 936, + -1, 938, -1, -1, 941, -1, 1564, -1, 172, -1, + 947, 948, -1, -1, -1, -1, 953, 2023, 2024, -1, + -1, -1, 959, 796, -1, 189, -1, -1, -1, 966, + 194, -1, 246, -1, -1, 3168, -1, -1, -1, 976, + 254, -1, -1, -1, 3191, -1, 983, -1, 3079, -1, + 3040, 3184, 266, -1, 3201, 3188, 993, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 3214, 232, -1, + -1, -1, -1, -1, 3079, -1, -1, -1, -1, -1, + -1, 3228, -1, 796, 298, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 1810, 503, -1, 3246, + 1870, 1871, 1872, 1873, 1874, 1875, 1043, 1317, 1878, 1879, + 1880, 1881, 1882, 1883, 1884, 1885, 1886, 1887, -1, -1, + -1, 285, -1, 3270, -1, -1, -1, -1, -1, -1, + -1, 2137, 217, 3228, -1, 349, 1346, 1347, -1, 3286, + 40, -1, 1700, 1857, -1, -1, -1, -1, 3295, -1, + 3297, 1088, 1362, 1090, 8, 1869, -1, 11, 372, -1, + -1, 15, 16, -1, -1, 19, 20, 21, 1105, 1106, + -1, 3161, -1, 3320, 1111, 3270, -1, 391, -1, 3312, + -1, -1, 36, -1, -1, 1899, -1, -1, 3335, 3336, + 3337, -1, 1906, -1, -1, -1, 1133, 3228, -1, -1, + 285, 101, -1, 103, -1, 105, 3339, 1144, 1145, -1, + -1, -1, -1, 986, -1, 3362, -1, 3291, 382, -1, + 1157, -1, -1, 3228, -1, 3320, -1, -1, -1, -1, + 1944, -1, -1, -1, 1948, -1, -1, 1174, -1, 3270, + 1954, 1178, 285, -1, -1, 1182, 0, -1, -1, -1, + -1, -1, 1810, 2023, 2024, -1, 3403, 167, -1, -1, + -1, -1, -1, 8, 18, 3270, 11, 21, -1, -1, + 15, 16, -1, 986, 19, 20, 21, -1, -1, -1, + -1, -1, 36, -1, -1, -1, -1, -1, -1, 3320, + -1, 45, -1, -1, -1, -1, -1, -1, -1, 1857, + -1, -1, -1, -1, -1, 3295, -1, -1, -1, -1, + -1, 1869, -1, -1, -1, 3320, -1, 71, 72, 73, + -1, -1, -1, 487, -1, -1, 490, 491, 492, -1, + 494, 495, 496, 497, 498, 499, -1, -1, -1, -1, + 94, 1899, 3332, -1, -1, -1, -1, -1, 1906, -1, + -1, -1, -1, -1, 1564, 26, -1, -1, -1, 107, + 108, 32, -1, 217, -1, -1, -1, -1, -1, 40, + -1, -1, -1, -1, 1311, -1, 1313, -1, -1, -1, + -1, 1318, -1, -1, -1, 1158, 1944, 297, -1, 60, + 1948, -1, 1329, -1, -1, -1, 1954, 307, 308, 309, + -1, -1, -1, -1, -1, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, -1, -1, -1, -1, -1, + -1, -1, 1359, -1, -1, -1, -1, 1364, 2434, 2435, + -1, 285, -1, 104, -1, -1, 184, 185, -1, -1, + -1, -1, -1, -1, -1, 1158, -1, 490, 491, 492, + -1, 494, 495, 496, 497, 498, 499, -1, 1395, -1, + 370, 1398, 133, 1400, 1401, -1, 1403, 1404, -1, -1, + -1, -1, 217, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 167, -1, + 1700, -1, -1, 1430, -1, -1, -1, 407, -1, -1, + 248, 249, 250, 251, 252, 253, -1, -1, 256, 257, + -1, -1, -1, -1, -1, 1452, 1453, -1, -1, -1, + 12, 1458, -1, 1460, -1, 17, 197, 2241, -1, -1, + 440, -1, -1, -1, -1, -1, 1473, 29, 1475, 1476, + 285, 451, -1, -1, 1317, -1, 1483, -1, 1485, -1, + 1487, 43, 44, 1490, 26, 465, -1, -1, 1495, -1, + 32, 1498, -1, 1500, -1, -1, -1, 1504, 40, 1506, + -1, 1508, -1, 1346, 1347, 246, -1, -1, -1, 489, + -1, -1, -1, 254, -1, -1, -1, -1, 60, 1362, + -1, -1, -1, 503, -1, 266, 1533, -1, -1, -1, + 1810, 1538, -1, -1, 1317, -1, -1, -1, -1, -1, + 102, -1, -1, -1, -1, -1, 287, -1, 518, 367, + 368, 2627, 522, -1, -1, 525, -1, 298, 307, 308, + 309, -1, 104, 1346, 1347, -1, 490, 491, 492, 1576, + 494, 495, 496, 497, 498, 499, 2652, 1857, -1, 1362, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 1869, + -1, 133, -1, -1, 2434, 2435, -1, -1, -1, -1, + 2676, -1, 343, -1, -1, -1, 347, -1, 349, -1, + -1, -1, -1, 2241, -1, -1, -1, -1, 1625, 1899, + 1627, 370, 1629, -1, -1, -1, 1906, -1, -1, -1, + -1, 372, 1639, 1640, -1, -1, 377, -1, 608, -1, + -1, -1, -1, -1, 55, 1652, 2430, -1, -1, -1, + 391, -1, -1, -1, -1, 197, -1, -1, 407, -1, + -1, -1, -1, -1, 1944, 483, 484, -1, 1948, -1, + 1677, -1, 1679, -1, 1954, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, 503, -1, 1694, 1695, 100, + 2766, 440, -1, 517, -1, -1, -1, -1, 1705, 1706, + 1707, 1708, 451, 1710, 246, -1, -1, 448, 119, 1716, + -1, 1718, 254, -1, -1, -1, 465, -1, -1, -1, + -1, 1564, 1729, 1730, 266, -1, 137, 1734, 1735, -1, + 141, 701, 1739, 1740, 1741, 1742, -1, 1744, 1745, -1, + 489, 565, 566, -1, -1, 287, -1, -1, -1, 2533, + -1, -1, 163, -1, 503, 166, 298, -1, -1, 1766, + 1767, 1768, 1769, 1770, -1, 735, 736, 737, 738, -1, + -1, 182, -1, -1, -1, -1, -1, 1784, -1, -1, + -1, 1564, 606, -1, -1, -1, -1, 2627, -1, -1, + -1, -1, -1, 617, -1, -1, -1, -1, -1, -1, + -1, 343, 2430, -1, -1, 347, -1, 349, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 0, -1, -1, + -1, 2605, -1, -1, -1, 2901, -1, -1, -1, -1, + 372, -1, -1, -1, -1, 377, -1, -1, -1, -1, + -1, 665, 37, -1, -1, 37, 2630, 42, -1, 391, + 42, 8, -1, -1, 11, -1, -1, 1700, 15, 16, + 684, -1, 273, 1870, 1871, 1872, 1873, 1874, 1875, -1, + 281, 1878, 1879, 1880, 1881, 1882, 1883, 1884, 1885, 1886, + 1887, -1, -1, -1, -1, -1, -1, -1, -1, 46, + -1, -1, 303, -1, -1, -1, 53, -1, -1, -1, + -1, 1908, -1, 2687, 99, 2533, 448, 99, -1, -1, + -1, -1, -1, 1920, -1, 98, -1, 1700, 329, -1, + -1, 78, -1, 1930, -1, -1, 2766, -1, -1, 1936, + -1, -1, -1, -1, -1, 1942, -1, -1, -1, -1, + 1947, -1, -1, 8, -1, -1, 11, -1, -1, -1, + 15, 16, -1, -1, -1, -1, -1, -1, -1, -1, + 143, 2241, -1, -1, -1, -1, -1, 1810, 938, -1, + 532, -1, -1, 168, 157, -1, 168, 2605, -1, 162, + -1, 46, -1, -1, 167, 1992, -1, -1, 53, 1996, + -1, -1, -1, 176, 2001, -1, -1, -1, 181, 561, + 562, -1, 2630, -1, -1, -1, -1, -1, -1, 204, + 2794, -1, 204, 78, 1857, 172, 2023, 2024, -1, -1, + -1, -1, -1, 3099, -1, 220, 1869, 1810, 220, -1, + 213, -1, 189, 2040, -1, 230, -1, 194, 230, -1, + -1, -1, -1, -1, 2828, -1, -1, -1, 107, 108, + 233, -1, -1, -1, -1, -1, 1899, -1, -1, 2687, + -1, 2901, -1, 1906, -1, -1, 2073, 262, -1, -1, + 262, -1, -1, -1, 1857, 232, 141, -1, 2085, -1, + -1, -1, -1, 2090, -1, -1, 1869, 2094, 283, -1, + -1, 283, -1, -1, -1, -1, -1, 280, -1, -1, + 283, 1944, 926, -1, -1, 1948, 289, 172, -1, -1, + -1, 1954, 307, -1, 676, 307, 1899, -1, 2902, 314, + -1, -1, 314, 1906, 189, 184, 185, 2134, 285, 194, + 2137, -1, -1, 2140, -1, 959, -1, -1, -1, -1, + -1, -1, 966, -1, -1, 328, -1, -1, -1, 2156, + 2430, -1, -1, -1, -1, -1, -1, -1, -1, 354, + -1, 1944, 354, -1, -1, 1948, 2794, 232, 351, -1, + -1, 1954, -1, -1, 1144, -1, 371, 1001, -1, 371, + -1, -1, -1, -1, -1, -1, -1, 1157, 1012, 248, + 249, 250, 251, 252, 253, -1, -1, 256, 257, -1, + 2828, -1, -1, -1, -1, -1, -1, -1, 1178, -1, + -1, -1, -1, 2220, -1, 1039, -1, -1, -1, 2226, + 285, -1, -1, -1, 407, 382, -1, -1, -1, -1, + 2237, 2238, 2239, 2240, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 2251, 440, 2253, -1, 440, 2256, + 445, -1, 435, 2533, 2261, -1, 3040, -1, -1, -1, + -1, -1, 445, -1, -1, -1, -1, 2274, -1, -1, + -1, -1, -1, -1, 2902, -1, -1, -1, -1, 462, + -1, 464, 465, -1, 2291, -1, -1, -1, -1, -1, + -1, -1, -1, 1117, 1118, -1, 1120, -1, -1, -1, + -1, -1, 2309, 2310, -1, 2312, -1, -1, 367, 368, + -1, -1, -1, -1, -1, -1, -1, 382, 501, -1, + -1, 504, 505, 506, -1, 2605, -1, -1, 0, -1, + 487, -1, -1, -1, 2341, -1, -1, 494, 495, 496, + 497, 498, 499, 1313, -1, -1, -1, -1, 1318, -1, + 2630, -1, -1, -1, 2361, -1, -1, -1, -1, -1, + -1, -1, 2369, 2370, 2371, -1, -1, -1, -1, -1, + -1, -1, 2379, -1, 2381, -1, 2383, -1, -1, -1, + -1, -1, 2389, -1, -1, -1, -1, -1, -1, -1, + -1, 953, -1, -1, 941, -1, -1, 2404, 2241, -1, + -1, -1, -1, -1, -1, -1, -1, 2687, -1, -1, + -1, -1, 3040, -1, 976, -1, -1, -1, -1, -1, + -1, -1, 487, -1, 483, 484, 98, 2434, 2435, 494, + 495, 496, 497, 498, 499, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 1896, -1, -1, 369, -1, -1, -1, 1903, -1, - -1, 231, -1, -1, -1, -1, -1, 216, -1, -1, - -1, 2791, -1, -1, -1, -1, 1397, 1398, -1, 1400, - -1, -1, -1, -1, 1308, -1, 1310, 284, -1, -1, - -1, 1315, -1, -1, 935, -1, 1941, -1, -1, -1, - 1945, -1, 1326, -1, -1, 2825, 1951, 380, -1, -1, - -1, -1, -1, -1, 284, -1, -1, 1114, 1115, -1, - 1117, -1, 437, -1, -1, -1, -1, 442, -1, -1, - -1, -1, 1356, -1, -1, 284, -1, 1361, -1, -1, - -1, -1, 2872, -1, -1, -1, 1807, -1, -1, -1, - 8, 1472, -1, 11, -1, -1, -1, 15, 16, -1, - -1, 19, 20, 21, -1, 2895, -1, -1, 1392, -1, - -1, 1395, -1, 1397, 1398, -1, 1400, 1401, 36, 2899, - -1, -1, 1503, 380, 1505, -1, -1, -1, -1, -1, - -1, -1, -1, 1854, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 1427, -1, 1866, -1, -1, -1, -1, - 380, 484, -1, -1, 487, 488, 489, -1, 491, 492, - 493, 494, 495, 496, -1, 1449, 1450, -1, -1, -1, - -1, 1455, -1, 1457, -1, 1896, -1, -1, -1, 2969, - -1, -1, 1903, -1, -1, -1, 1470, -1, 1472, 1473, - -1, -1, -1, -1, -1, -1, 1480, -1, 1482, -1, - 1484, -1, -1, 1487, -1, -1, -1, -1, 1492, -1, - -1, 1495, -1, 1497, -1, -1, -1, 1501, -1, 1503, - 1941, 1505, -1, -1, 1945, -1, -1, 484, -1, -1, - 1951, -1, -1, 167, 491, 492, 493, 494, 495, 496, - 1141, -1, -1, -1, -1, -1, 1530, 3037, 167, 920, - -1, 1535, -1, 1154, 484, -1, -1, 3037, -1, -1, - -1, 491, 492, 493, 494, 495, 496, -1, -1, -1, - -1, -1, -1, -1, 1175, -1, -1, -1, 487, 488, - 489, -1, 491, 492, 493, 494, 495, 496, -1, 1573, - -1, -1, -1, -1, -1, 1676, -1, -1, 216, -1, + -1, -1, -1, -1, -1, -1, 2463, -1, 2241, -1, + -1, -1, 2469, -1, -1, -1, -1, -1, -1, -1, + -1, 143, -1, -1, -1, -1, -1, -1, 2485, -1, + -1, 1043, 2489, -1, -1, 157, 2493, 2494, 2495, -1, + 162, -1, 2499, 2500, 2501, 167, 2503, -1, -1, -1, + -1, -1, -1, -1, 176, -1, -1, -1, -1, 181, + -1, 3295, -1, 1483, 2794, 2522, -1, 2524, 8, -1, + -1, 11, -1, -1, -1, 15, 16, -1, -1, -1, + 2537, 2538, 2539, 2540, 2541, 2542, 2543, 2544, 2545, 2546, + -1, 213, -1, 1105, 1106, -1, -1, -1, 2828, -1, + -1, -1, -1, -1, -1, -1, 46, -1, 2565, -1, + -1, 233, -1, 53, 2571, -1, -1, -1, -1, 923, + -1, -1, -1, -1, 2581, -1, -1, -1, 37, -1, + -1, -1, -1, 42, -1, 1409, 1133, 2430, 78, -1, + 2597, -1, 8, -1, 24, 11, -1, -1, -1, 15, + 16, -1, -1, 19, 20, 21, -1, -1, 280, 2616, + -1, 283, -1, -1, -1, -1, -1, 289, -1, -1, + 2627, -1, 2902, -1, -1, -1, 2633, -1, -1, -1, + 984, -1, -1, 2640, 2641, 2642, 2643, -1, -1, -1, + 99, -1, -1, -1, -1, 2652, -1, 2430, 2655, 79, + -1, 141, 2659, 2660, -1, -1, 328, 1627, -1, 1629, + -1, 1485, -1, 1487, -1, 95, 1490, 3295, 2675, 1639, + -1, 1495, -1, -1, 1498, -1, 1500, -1, -1, 351, + 1504, -1, 172, -1, -1, -1, -1, -1, -1, -1, + 2533, -1, -1, -1, -1, -1, -1, 1521, 2705, 189, + 2707, -1, -1, 1057, 194, -1, -1, 1677, -1, 168, + -1, -1, 142, -1, 2721, -1, -1, 176, 2725, -1, + -1, -1, 152, -1, 2731, -1, -1, -1, -1, -1, + 1084, -1, 1086, 1087, 164, 407, -1, -1, 2745, 169, + -1, -1, 232, -1, 2751, 204, 8, -1, -1, 11, + 2533, -1, -1, 15, 16, 1579, -1, 2764, 2765, 2766, + 3040, 220, 2605, 435, -1, 2772, -1, -1, 198, -1, + -1, 230, -1, 445, -1, -1, -1, -1, -1, -1, + -1, -1, 2789, -1, 46, -1, -1, 2630, -1, -1, + 462, 53, 464, 465, -1, 285, 1766, 1767, 1768, 1769, + 1770, 217, -1, 262, -1, 1159, -1, -1, 1162, 1163, + -1, 241, -1, -1, -1, 245, 78, -1, -1, -1, + -1, -1, 2605, -1, 283, 1649, -1, -1, -1, 501, + -1, -1, 504, 505, 506, -1, -1, -1, 1400, 1401, + -1, 1403, -1, -1, 2687, -1, -1, 2630, 307, -1, + -1, -1, -1, -1, -1, 314, -1, -1, -1, 2866, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 285, + -1, 2878, -1, -1, -1, -1, -1, -1, -1, 141, + -1, -1, 312, -1, -1, 2892, -1, -1, -1, -1, + -1, -1, 382, -1, 2901, 354, -1, 327, -1, -1, + -1, -1, -1, -1, 2687, 1452, 1453, -1, -1, -1, + 172, 1458, 371, 1475, -1, -1, -1, -1, -1, -1, + 2927, -1, -1, -1, -1, -1, 2933, 189, -1, -1, + 360, -1, 194, 363, 2941, -1, -1, -1, -1, -1, + -1, -1, 372, -1, 1506, 375, 1508, -1, -1, -1, + 1920, 2794, -1, 1777, -1, -1, -1, -1, -1, -1, + -1, 2968, 1786, 393, -1, -1, -1, -1, -1, -1, + 232, -1, -1, -1, -1, -1, -1, 407, -1, -1, + -1, 440, -1, 2990, 414, 2828, 445, -1, 2995, -1, + 1344, -1, -1, -1, 424, -1, -1, 487, -1, -1, + 430, 1355, -1, 1357, 494, 495, 496, 497, 498, 499, + -1, 2794, -1, 1367, 3021, 3295, -1, -1, -1, -1, + -1, -1, -1, 285, -1, -1, -1, 457, -1, -1, + 1384, -1, -1, -1, 3041, 3042, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 2828, -1, -1, -1, 3056, + -1, 3058, -1, -1, -1, 3062, -1, -1, 1605, 2902, + -1, 1415, 1416, -1, -1, -1, -1, -1, -1, 3076, + -1, -1, 3079, -1, 490, 491, 492, -1, 494, 495, + 496, 497, 498, 499, -1, -1, -1, -1, -1, -1, + -1, -1, 3099, -1, -1, -1, 3103, 3104, -1, -1, + 3107, -1, -1, -1, -1, 1652, 1930, -1, -1, -1, + -1, -1, 1936, -1, -1, -1, -1, 1679, -1, 2902, + 382, -1, -1, 3130, -1, -1, -1, -1, -1, -1, + 3137, -1, -1, 1695, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 1705, -1, 1707, -1, -1, 1710, -1, + -1, -1, -1, -1, 1716, -1, 1718, -1, -1, -1, + 3167, -1, -1, -1, -1, -1, -1, -1, 1730, -1, + -1, -1, -1, 1735, -1, -1, -1, 1739, 1740, 1741, + 1742, -1, 1744, 1745, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 3040, 3205, -1, + -1, -1, -1, -1, 2028, -1, -1, -1, -1, -1, + -1, 652, 653, -1, -1, -1, -1, -1, -1, -1, + -1, 3228, 2046, -1, -1, 487, -1, -1, -1, -1, + -1, -1, 494, 495, 496, 497, 498, 499, 2062, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 3255, 2073, + -1, -1, -1, -1, -1, -1, 2226, 3040, -1, -1, + 3267, 3268, -1, 3270, 3271, -1, 2090, -1, 2238, 2239, + 2240, -1, 3279, -1, 0, -1, 717, -1, -1, -1, + 721, -1, -1, 2253, 3291, -1, 2256, -1, -1, -1, + -1, 2261, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 3313, -1, -1, 1663, + 1664, -1, -1, 3320, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 1870, 1871, 1872, 1873, 1874, 1875, -1, + -1, 1878, 1879, 1880, 1881, 1882, 1883, 1884, 1885, 1886, + 1887, -1, -1, -1, -1, -1, 1908, -1, -1, -1, + 8, 3358, -1, 11, -1, -1, -1, 15, 16, 17, + 18, 19, 20, 21, -1, -1, 3373, -1, -1, 1723, + -1, -1, 98, -1, -1, -1, -1, -1, 36, 1733, + 1942, -1, 1736, -1, 8, 1947, -1, 11, 46, -1, + -1, 15, 16, 834, -1, 53, -1, -1, -1, 2369, + 2370, 2371, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 2235, -1, -1, -1, -1, -1, 143, -1, -1, + 78, -1, 46, -1, -1, -1, -1, -1, -1, 53, + 1992, 157, -1, -1, -1, -1, 162, -1, -1, 2001, + -1, 167, -1, -1, -1, -1, -1, -1, -1, -1, + 176, -1, 3295, -1, 78, 181, 8, -1, -1, 11, + 1814, 1815, -1, 15, 16, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 2023, 2024, -1, -1, + -1, -1, -1, -1, -1, -1, 1840, 213, -1, -1, + 931, -1, -1, -1, 46, -1, 937, -1, -1, 940, + -1, 53, 943, 944, 945, 946, -1, 233, -1, -1, + -1, -1, 3295, -1, 172, 2485, -1, 141, -1, -1, + -1, -1, -1, 2085, -1, -1, 78, -1, -1, -1, + -1, 189, -1, -1, -1, -1, 194, -1, -1, -1, + -1, -1, -1, -1, 985, -1, -1, -1, 172, -1, + -1, -1, -1, -1, 280, -1, -1, 283, -1, 217, + 218, -1, -1, 289, 1005, 189, -1, -1, -1, -1, + 194, -1, 2134, -1, 232, -1, -1, -1, 2140, -1, + -1, -1, 1023, -1, 2408, -1, -1, -1, -1, 141, + 2137, -1, -1, 1034, 1035, 1036, 1950, 1038, 1039, -1, + -1, 2571, 328, -1, -1, -1, -1, -1, 232, -1, + 268, -1, -1, 271, -1, -1, 2440, -1, -1, -1, + 172, -1, -1, -1, -1, 351, -1, 285, 1069, -1, + 288, -1, -1, -1, -1, 2459, -1, 189, -1, -1, + -1, 2465, 194, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 981, 1692, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 1702, -1, 1704, -1, -1, 1707, -1, -1, -1, - -1, -1, 1713, 2238, 1715, -1, 8, -1, 1622, 11, - 1624, -1, 1626, 15, 16, -1, 1727, -1, -1, 1406, - -1, 1732, 1636, 1637, -1, 1736, 1737, 1738, 1739, -1, - 1741, 1742, -1, -1, -1, 1649, 284, -1, 3158, -1, - -1, 305, 306, 307, 46, -1, 295, -1, -1, -1, - -1, 53, -1, 1054, -1, -1, 305, 306, 307, -1, - 1674, -1, 1676, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 78, 1691, 1692, 1310, - 1081, -1, 1083, 1084, 1315, -1, -1, -1, 1702, 1703, - 1704, 1705, -1, 1707, -1, 1482, -1, 1484, -1, 1713, - 1487, 1715, -1, -1, 368, 1492, -1, -1, 1495, -1, - 1497, -1, 1726, 1727, 1501, -1, -1, 1731, 1732, 368, - -1, -1, 1736, 1737, 1738, 1739, -1, 1741, 1742, -1, - -1, 1518, -1, -1, -1, -1, -1, -1, -1, 141, - -1, 405, -1, -1, -1, -1, -1, -1, -1, 1763, - 1764, 1765, 1766, 1767, -1, 1156, 405, -1, 1159, 1160, - -1, -1, -1, -1, -1, -1, -1, 1781, -1, -1, - 172, -1, 3292, 437, -1, -1, -1, -1, -1, -1, - -1, -1, 3292, -1, 448, -1, 188, 2238, 437, 1576, - -1, 193, 2427, -1, 1905, -1, -1, -1, 462, 448, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 3329, - -1, -1, -1, 462, -1, -1, -1, -1, -1, -1, - -1, -1, 486, -1, -1, -1, -1, -1, 1939, 231, - -1, -1, -1, 1944, -1, -1, 500, 486, -1, 487, - 488, 489, -1, 491, 492, 493, 494, 495, 496, 1480, - -1, 500, -1, 1867, 1868, 1869, 1870, 1871, 1872, 1646, - -1, 1875, 1876, 1877, 1878, 1879, 1880, 1881, 1882, 1883, - 1884, -1, -1, -1, -1, 0, -1, -1, 1989, -1, - -1, -1, 284, -1, -1, -1, -1, 1998, -1, -1, - -1, 1905, 938, -1, -1, 2530, -1, 40, -1, -1, - -1, -1, -1, 1917, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 1927, -1, -1, -1, 60, -1, 1933, - -1, -1, -1, -1, -1, 1939, -1, -1, -1, -1, - 1944, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 1341, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 1352, -1, 1354, -1, -1, -1, -1, -1, -1, - -1, 104, 105, 1364, -1, -1, -1, 2602, -1, -1, - 113, 2082, -1, 98, -1, 1989, 2427, -1, 380, 1993, - 1381, -1, -1, -1, 1998, -1, -1, 1774, -1, -1, - -1, -1, 2627, 1624, -1, 1626, 1783, -1, -1, -1, - -1, -1, -1, -1, -1, 1636, 2020, 2021, -1, -1, - -1, 1412, 1413, -1, -1, -1, 37, -1, 143, -1, - 2131, 42, -1, 2037, -1, -1, 2137, 8, 171, -1, - 11, -1, 157, -1, 15, 16, -1, 162, -1, -1, - -1, -1, 167, 1674, -1, -1, -1, -1, -1, 2684, - -1, 176, -1, 196, -1, 180, 2070, -1, -1, -1, - -1, -1, -1, -1, -1, 46, -1, -1, 2082, -1, - -1, -1, 53, 2087, -1, -1, -1, 2091, 99, 2530, - -1, -1, 484, -1, 1130, -1, -1, 212, -1, 491, - 492, 493, 494, 495, 496, -1, -1, 78, -1, -1, - -1, -1, 245, -1, -1, -1, -1, 232, -1, -1, - 253, -1, -1, -1, -1, -1, -1, 2131, -1, -1, - 2134, -1, 265, 2137, 267, -1, -1, -1, -1, -1, - -1, -1, 1763, 1764, 1765, 1766, 1767, 2248, -1, 2153, - 1927, -1, -1, -1, -1, -1, 1933, 168, -1, -1, - -1, 2602, -1, 296, 279, -1, 2791, 282, -1, -1, - 141, 8, -1, 288, 11, -1, -1, -1, 15, 16, - -1, -1, -1, -1, -1, -1, 2627, -1, -1, -1, - -1, -1, 203, -1, -1, -1, -1, -1, -1, -1, - 2825, 172, -1, -1, -1, -1, -1, -1, 219, 46, - -1, 326, -1, 2217, 347, -1, 53, 188, 229, 2223, - -1, -1, 193, -1, -1, -1, -1, -1, -1, -1, - 2234, 2235, 2236, 2237, 349, -1, -1, 370, -1, -1, - -1, 78, -1, 2684, 2248, -1, 2250, -1, 2025, 2253, - 261, -1, -1, -1, 2258, -1, 389, -1, 391, -1, - 231, 394, -1, -1, -1, -1, 2043, 2271, -1, 1660, - 1661, 282, -1, -1, 2899, 2376, -1, -1, -1, 2380, - -1, -1, 2059, -1, 2288, 2386, -1, -1, -1, -1, - 405, -1, -1, 2070, 305, -1, 1917, -1, -1, -1, - -1, 312, 2306, 2307, 141, 2309, -1, -1, -1, -1, - 2087, -1, -1, 284, -1, -1, -1, 432, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 442, -1, 1720, - -1, -1, -1, -1, 2338, 172, -1, -1, -1, 1730, - -1, 352, 1733, -1, 459, -1, 461, 462, -1, -1, - 2791, 188, -1, -1, 2358, -1, 193, -1, 369, -1, - -1, -1, 2366, 2367, 2368, -1, -1, 500, -1, -1, - -1, -1, 2376, -1, 2378, -1, 2380, -1, -1, -1, - -1, -1, 2386, 498, 2825, 2486, 501, 502, 503, 2490, - 2491, 2492, -1, -1, 231, -1, -1, 2401, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 380, - -1, -1, 3037, 1449, 1450, -1, -1, -1, 2519, 1455, - 1811, 1812, -1, -1, -1, -1, 437, 2431, 2432, -1, - -1, -1, -1, 2534, 2535, 2536, 2537, 2538, 2539, 2540, - 2541, 2542, 2543, -1, -1, -1, 1837, 284, -1, -1, - -1, -1, -1, -1, -1, 2232, 2460, -1, 2899, -1, - -1, -1, 2466, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 2482, -1, - -1, -1, 2486, -1, -1, -1, 2490, 2491, 2492, -1, - -1, -1, 2496, 2497, 2498, -1, 2500, -1, -1, -1, + -1, 285, -1, -1, -1, -1, -1, 2491, -1, -1, + 2640, 2641, 2642, 2643, -1, -1, -1, -1, 1119, -1, + 232, 407, 1123, 1124, -1, -1, -1, -1, -1, 2251, + -1, -1, -1, 1134, 1135, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 435, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 445, + -1, -1, -1, 2077, 382, -1, -1, -1, -1, -1, + -1, -1, -1, 285, -1, -1, 462, -1, 464, 465, + -1, -1, 2566, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 2110, 2111, 382, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 2592, -1, + -1, 2595, -1, -1, -1, 501, -1, -1, 504, 505, + 506, -1, -1, -1, -1, -1, 8, -1, -1, 11, + -1, -1, -1, 15, 16, 17, 18, 19, 20, 21, + -1, -1, 2772, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 36, -1, -1, 2379, -1, -1, + -1, 2383, -1, -1, 46, -1, 1267, 2389, 2652, 487, + 382, 53, 490, 491, 492, 1276, 494, 495, 496, 497, + 498, 499, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 513, 78, -1, -1, 1300, + -1, -1, -1, 487, -1, -1, -1, -1, -1, -1, + 494, 495, 496, 497, 498, 499, -1, -1, -1, -1, + -1, -1, -1, 2707, -1, -1, -1, 2434, 2435, -1, + -1, -1, -1, -1, -1, -1, -1, 2721, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 484, -1, 2519, -1, 2521, -1, -1, - 491, 492, 493, 494, 495, 496, -1, -1, -1, 2630, - 2534, 2535, 2536, 2537, 2538, 2539, 2540, 2541, 2542, 2543, - -1, -1, -1, 380, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 2656, 1947, -1, 2562, -1, - -1, -1, -1, -1, 2568, -1, 1602, -1, -1, -1, - -1, -1, -1, 8, 2578, -1, 11, -1, -1, -1, - 15, 16, -1, -1, 19, 20, 21, -1, -1, -1, - 2594, -1, -1, -1, -1, -1, 3037, -1, -1, -1, - -1, 36, 2223, -1, -1, -1, -1, -1, -1, 2613, - -1, 649, 650, 1649, 2235, 2236, 2237, -1, -1, -1, - 2624, -1, -1, -1, -1, -1, 2630, -1, 2405, 2250, - -1, -1, 2253, 2637, 2638, 2639, 2640, 2258, -1, -1, - -1, -1, 0, -1, -1, 2649, -1, 484, 2652, -1, - -1, -1, 2656, 2657, 491, 492, 493, 494, 495, 496, - 2437, 2762, -1, -1, -1, -1, -1, 3292, 2672, -1, - -1, -1, -1, -1, -1, -1, 714, -1, -1, 2456, - 718, -1, -1, 2074, 8, 2462, -1, 11, -1, -1, - -1, 15, 16, -1, -1, 19, 20, 21, 2702, -1, - 2704, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 2488, 36, -1, 2718, -1, 2107, 2108, 2722, -1, - 8, -1, -1, 11, 2728, -1, -1, 15, 16, -1, - -1, 19, 20, 21, -1, -1, -1, -1, 2742, -1, - 98, -1, -1, -1, 2748, 2366, 2367, 2368, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 2761, 2762, 2763, - -1, -1, -1, -1, -1, 2769, 8, -1, -1, 11, - -1, -1, -1, 15, 16, -1, -1, 19, 20, 21, - -1, 216, 2786, -1, -1, 143, 2563, -1, -1, -1, - -1, -1, -1, 831, -1, -1, -1, -1, -1, 157, - -1, -1, -1, -1, 162, -1, -1, -1, -1, 167, - -1, -1, 2589, -1, -1, 2592, -1, -1, 176, -1, - -1, -1, 180, -1, -1, -1, -1, -1, -1, 2930, - -1, 1867, 1868, 1869, 1870, 1871, 1872, 2938, -1, 1875, - 1876, 1877, 1878, 1879, 1880, 1881, 1882, 1883, 1884, 284, - -1, 3292, -1, -1, 212, -1, -1, -1, -1, 2863, - -1, 2482, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 2875, 2649, 2264, 232, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 2889, 2987, -1, -1, -1, - 928, 2992, 216, -1, 2898, -1, 934, -1, -1, 937, - -1, -1, 940, 941, 942, 943, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 3018, -1, -1, - 2924, 279, -1, -1, 282, -1, 2930, 2704, 216, -1, - 288, -1, -1, -1, 2938, -1, -1, 3038, 3039, -1, - -1, 2718, -1, -1, 982, -1, -1, 2568, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 3059, -1, - 284, 2965, -1, -1, 1002, -1, -1, -1, 326, -1, - -1, -1, -1, -1, 216, -1, -1, -1, -1, -1, - -1, -1, 1020, 2987, 2020, 2021, -1, -1, 2992, -1, - -1, 349, -1, 1031, 1032, 1033, 284, 1035, 1036, 3100, + -1, -1, -1, 2267, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 2489, -1, -1, + -1, 2493, 2494, 2495, -1, 487, -1, -1, -1, -1, + -1, -1, 494, 495, 496, 497, 498, 499, -1, -1, + 172, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 2522, -1, -1, -1, -1, -1, -1, 189, -1, -1, + -1, -1, 194, -1, -1, 2537, 2538, 2539, 2540, 2541, + 2542, 2543, 2544, 2545, 2546, -1, -1, -1, -1, -1, + -1, -1, 1433, -1, -1, 217, 218, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3018, -1, 2637, 2638, 2639, 2640, - -1, -1, -1, -1, -1, -1, -1, -1, 1066, -1, - -1, -1, -1, -1, 3038, 3039, -1, -1, -1, -1, - -1, -1, 284, -1, -1, -1, -1, 405, -1, 3053, - -1, 3055, 487, 488, 489, 3059, 491, 492, 493, 494, - 495, 496, -1, -1, -1, -1, -1, -1, -1, 3073, - 107, 108, 3076, -1, 432, -1, -1, -1, 1116, -1, - -1, -1, 1120, 1121, 442, -1, -1, -1, -1, -1, - -1, -1, 3096, 1131, 1132, -1, 3100, 3101, 2134, 2876, - 3104, 459, -1, 461, 462, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 8, -1, -1, 11, -1, -1, - -1, 15, 16, 3127, -1, 19, 20, 21, -1, -1, - 3134, -1, -1, -1, -1, -1, -1, 2914, -1, 2530, - 498, -1, 36, 501, 502, 503, 183, 184, 2769, -1, - -1, -1, 46, -1, -1, -1, -1, -1, -1, 53, - 3164, -1, -1, 487, 488, 489, -1, 491, 492, 493, - 494, 495, 496, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 78, -1, -1, 3288, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 3202, 487, - 488, 489, -1, 491, 492, 493, 494, 495, 496, -1, - 247, 248, 249, 250, 251, 252, -1, -1, 255, 256, - -1, 3225, -1, -1, -1, -1, 1264, -1, -1, -1, - -1, -1, -1, -1, -1, 1273, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 487, 488, 489, 3252, 491, - 492, 493, 494, 495, 496, -1, -1, -1, -1, 1297, - 3264, 3265, -1, 3267, 3268, -1, -1, -1, -1, -1, - -1, -1, 3276, 3050, 3051, -1, -1, -1, 172, -1, - -1, -1, -1, -1, 3288, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 188, -1, -1, -1, -1, 193, - 8, -1, -1, 11, -1, -1, 3310, 15, 16, -1, - -1, -1, -1, 3317, -1, -1, -1, -1, -1, -1, - -1, -1, 216, 217, -1, -1, -1, -1, 365, 366, - -1, -1, -1, -1, -1, -1, -1, 231, 46, -1, - -1, -1, -1, -1, -1, 53, -1, 3124, -1, 3126, - 3354, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3368, -1, -1, -1, -1, -1, - 78, -1, -1, 267, -1, -1, 270, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 3165, 2780, - 284, -1, 1430, 287, 2785, 2431, 2432, -1, -1, -1, - -1, -1, -1, -1, 3181, -1, -1, -1, 3185, -1, + 232, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 1465, -1, 1467, 1468, -1, -1, + 8, -1, -1, 11, -1, -1, -1, 15, 16, 1480, + 1481, 19, 20, 21, -1, -1, 268, -1, -1, 271, + -1, -1, -1, -1, -1, 2879, 1497, -1, 36, -1, + -1, -1, -1, 285, -1, -1, 288, -1, 46, -1, + -1, 2633, -1, -1, -1, 53, -1, -1, -1, -1, + 2627, -1, -1, -1, -1, -1, 3056, -1, 1529, -1, + 1531, -1, -1, 2917, -1, -1, -1, 2659, -1, -1, + 78, -1, -1, -1, -1, 2652, 3076, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 1559, -1, + -1, -1, -1, -1, 1565, 1566, 1567, 1568, 1569, 1570, + 1571, 1572, -1, -1, -1, -1, 1577, 1578, -1, -1, + -1, 1582, -1, -1, -1, 1586, -1, -1, 1589, 1590, + 1591, 1592, 1593, 1594, 1595, 1596, 1597, -1, -1, 1600, + 382, -1, -1, -1, -1, -1, 1607, 3137, 1609, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 2533, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 1630, + -1, -1, -1, -1, 172, -1, -1, 3167, -1, -1, + -1, -1, -1, 2765, -1, -1, -1, -1, -1, -1, + -1, 189, -1, -1, -1, -1, 194, -1, -1, 2766, + -1, -1, 1663, 1664, -1, -1, 941, -1, -1, 3053, + 3054, -1, -1, -1, -1, -1, -1, -1, -1, 217, + 218, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 232, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 487, -1, -1, 490, 491, + 492, -1, 494, 495, 496, 497, 498, 499, -1, -1, + -1, -1, -1, -1, -1, 507, -1, -1, -1, -1, + 268, -1, -1, 271, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 3127, -1, 3129, -1, 285, 1749, -1, + 288, -1, 1753, -1, -1, 1756, 1757, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 1462, -1, 1464, 1465, -1, -1, - -1, -1, 3053, -1, -1, -1, -1, -1, -1, 1477, - 1478, 2832, 2833, 480, 481, -1, -1, -1, -1, -1, - -1, -1, 3073, -1, -1, -1, 1494, -1, -1, -1, - -1, -1, -1, 500, 172, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 188, -1, -1, -1, -1, 193, 380, -1, 1526, -1, - 1528, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 1058, 3168, -1, -1, -1, -1, 1790, + -1, -1, -1, -1, 2901, -1, -1, -1, -1, -1, + 3184, -1, 8, -1, 3188, 11, -1, -1, -1, 15, + 16, 2933, -1, 19, 20, 21, -1, -1, -1, 2941, + -1, -1, -1, -1, -1, -1, 1101, -1, -1, -1, + 1831, 1832, -1, -1, -1, -1, -1, -1, -1, -1, + 46, -1, -1, -1, 382, -1, 8, 53, -1, 11, + -1, -1, -1, 15, 16, -1, -1, -1, 1133, -1, + 1861, 1862, -1, 1864, -1, -1, -1, -1, 2990, 2783, + -1, -1, 78, 2995, 2788, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 46, -1, -1, -1, -1, -1, + -1, 53, 1893, 1894, -1, -1, 1897, -1, -1, 3021, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 1184, + -1, -1, -1, -1, -1, -1, 78, -1, -1, 3041, + 3042, 2835, 2836, -1, -1, 1926, -1, -1, 3312, -1, + -1, 1932, -1, -1, -1, -1, -1, -1, -1, -1, + 3062, -1, -1, -1, -1, -1, -1, -1, -1, 487, + -1, 1952, 490, 491, 492, 3339, 494, 495, 496, 497, + 498, 499, 1963, -1, -1, -1, 172, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 141, + -1, 3103, -1, 189, -1, -1, -1, -1, 194, -1, + -1, -1, 3099, 1994, -1, -1, -1, -1, 1999, 2000, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 3134, -1, -1, -1, -1, 1556, -1, - -1, -1, -1, 231, 1562, 1563, 1564, 1565, 1566, 1567, - 1568, 1569, 3309, -1, -1, -1, 1574, 1575, -1, -1, - -1, 1579, -1, 3164, -1, 1583, -1, -1, 1586, 1587, - 1588, 1589, 1590, 1591, 1592, 1593, 1594, -1, 3335, 1597, - -1, -1, -1, -1, -1, -1, 1604, -1, 1606, -1, - -1, -1, -1, -1, -1, -1, 284, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 2624, 1627, - 484, -1, -1, 487, 488, 489, -1, 491, 492, 493, - 494, 495, 496, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 2649, -1, -1, -1, -1, -1, -1, - -1, -1, 1660, 1661, -1, -1, -1, -1, -1, -1, - 0, -1, -1, -1, -1, 3026, 3027, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 3037, -1, -1, -1, - -1, -1, 22, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 32, -1, 34, 35, -1, -1, -1, -1, - -1, -1, 380, -1, -1, -1, -1, 47, -1, -1, - -1, -1, 52, -1, -1, -1, -1, -1, -1, -1, - -1, 61, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 73, -1, -1, -1, -1, 1746, -1, - -1, -1, 1750, -1, 84, 1753, 1754, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 96, 2763, 98, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 938, 109, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 1787, - -1, -1, -1, 123, 124, -1, -1, -1, -1, -1, - -1, -1, -1, 133, -1, -1, -1, -1, 26, 139, - -1, -1, -1, -1, 32, -1, 484, 147, -1, 149, - 150, -1, 40, 491, 492, 493, 494, 495, 496, -1, - 1828, 1829, -1, 163, -1, -1, -1, 167, -1, -1, - -1, -1, 60, -1, -1, -1, 8, -1, -1, 11, - -1, -1, -1, 15, 16, 17, 18, 19, 20, 21, - 1858, 1859, 192, 1861, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 36, -1, -1, -1, 208, -1, - -1, -1, -1, -1, 46, -1, 104, -1, -1, -1, - -1, 53, 1890, 1891, -1, 1055, 1894, -1, -1, -1, - -1, -1, 2898, -1, 234, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 133, 78, -1, -1, -1, - -1, -1, -1, -1, -1, 1923, -1, -1, -1, -1, - -1, 1929, -1, -1, -1, -1, -1, -1, 1098, -1, - -1, 3292, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 1949, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 1960, -1, -1, -1, -1, -1, -1, -1, - 1130, -1, -1, -1, -1, 305, 306, 307, 196, -1, - -1, -1, -1, 313, -1, -1, 316, -1, -1, -1, - -1, -1, -1, 1991, -1, -1, -1, -1, 1996, 1997, + 172, 217, 218, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 232, 189, 1303, -1, + -1, -1, 194, -1, -1, 3, -1, -1, -1, -1, + 8, 2042, -1, 11, 2045, -1, 2047, 15, 16, 17, + 18, 19, 20, 21, -1, -1, -1, -1, -1, -1, + -1, -1, 2063, -1, -1, 271, -1, -1, 36, -1, + 232, -1, 40, -1, -1, -1, -1, -1, 46, 285, + -1, -1, -1, -1, -1, 53, -1, -1, -1, -1, + -1, -1, 8, -1, -1, 11, -1, -1, -1, 15, + 16, -1, 2103, 19, 20, 21, -1, -1, -1, -1, + 78, -1, -1, -1, -1, 3029, 3030, -1, -1, 2120, + 2121, -1, -1, 285, -1, -1, 3040, -1, -1, -1, + 46, -1, 1407, -1, -1, -1, -1, 53, 2139, -1, + -1, -1, -1, -1, -1, -1, -1, 2148, -1, -1, + -1, -1, -1, -1, -1, -1, 1431, -1, -1, -1, + -1, -1, 78, -1, -1, -1, -1, -1, -1, 3291, + -1, -1, -1, 1448, -1, 1450, 382, 1452, 1453, -1, + 1455, -1, -1, 1458, -1, -1, 1461, -1, -1, 1464, + -1, -1, -1, -1, 1469, -1, -1, 1472, -1, -1, + -1, -1, -1, 2204, 172, -1, -1, -1, -1, 2210, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 172, -1, -1, -1, -1, 345, -1, -1, -1, -1, - -1, 1181, -1, -1, 354, -1, 188, 245, -1, -1, - -1, 193, -1, -1, -1, 253, -1, -1, 368, -1, - -1, 2039, -1, -1, 2042, 375, 2044, 265, -1, 379, - -1, -1, -1, -1, 216, 217, -1, -1, -1, 389, - -1, -1, 2060, -1, -1, -1, -1, -1, 286, 231, - -1, 401, -1, -1, -1, 405, -1, -1, 296, -1, + 382, 189, -1, -1, -1, -1, 194, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 424, -1, -1, -1, 24, -1, - 3096, -1, 2100, -1, -1, 267, -1, 437, 270, -1, - 440, -1, -1, 443, -1, -1, -1, -1, 448, 2117, - 2118, -1, 284, 341, -1, 287, -1, 345, -1, 347, - -1, -1, 462, -1, -1, -1, -1, -1, 2136, -1, - 1300, -1, -1, -1, -1, -1, -1, 2145, -1, -1, - -1, -1, 370, 79, -1, -1, 486, 375, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 95, - 500, 389, 8, 503, -1, 11, -1, -1, -1, 15, - 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 217, + 218, -1, 1527, -1, -1, -1, 172, 1532, 2259, -1, + 1535, 1536, 1537, -1, 232, -1, 1541, -1, -1, -1, + 1545, -1, -1, 189, -1, -1, -1, -1, 194, -1, + -1, 487, -1, -1, 490, 491, 492, -1, 494, 495, + 496, 497, 498, 499, -1, -1, -1, -1, -1, -1, + 268, 217, 218, 271, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 232, 285, -1, -1, + 288, -1, -1, -1, -1, 487, -1, -1, -1, -1, + -1, 1606, 494, 495, 496, 497, 498, 499, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 36, -1, -1, 2201, -1, -1, -1, -1, -1, 2207, - 46, -1, -1, -1, -1, -1, 142, 53, 380, -1, - -1, -1, -1, -1, -1, -1, 152, 445, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 164, -1, - -1, -1, 78, 169, 1404, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 2256, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 1428, -1, - -1, 197, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 1445, -1, 1447, -1, 1449, - 1450, -1, 1452, -1, -1, 1455, -1, -1, 1458, -1, - -1, 1461, -1, -1, -1, -1, 1466, -1, -1, 1469, - -1, -1, -1, -1, 240, -1, -1, -1, 244, -1, - -1, -1, 484, -1, -1, 487, 488, 489, -1, 491, - 492, 493, 494, 495, 496, -1, 172, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 510, -1, - -1, -1, 188, -1, -1, -1, -1, 193, -1, -1, - -1, -1, -1, -1, 1524, -1, -1, -1, -1, 1529, - -1, -1, 1532, 1533, 1534, -1, -1, -1, 1538, -1, - 216, 217, 1542, -1, 310, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 231, 2394, -1, -1, 325, - 2398, 2399, -1, -1, -1, -1, -1, 2405, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 2415, -1, -1, - 2418, -1, 2420, -1, -1, -1, -1, -1, -1, -1, - 2428, 267, 358, -1, 270, 361, -1, -1, 2436, 2437, - -1, -1, -1, 1603, 370, 2443, -1, 373, 284, -1, - -1, 287, -1, -1, -1, -1, -1, -1, -1, -1, - 2458, -1, -1, -1, -1, 391, -1, -1, -1, -1, - 2468, -1, -1, -1, -1, -1, -1, -1, -1, 405, - -1, -1, -1, 1643, -1, -1, 412, -1, -1, -1, - 2488, -1, -1, -1, -1, 421, -1, -1, -1, 1659, - -1, 427, -1, -1, 1664, -1, -1, -1, -1, -1, + -1, -1, 268, -1, -1, 271, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 285, + -1, 1646, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 3295, -1, -1, -1, -1, -1, 1662, -1, -1, + -1, -1, 1667, -1, -1, -1, 2397, -1, -1, -1, + 2401, 2402, -1, -1, -1, -1, -1, 2408, -1, 1684, + -1, -1, -1, -1, 382, -1, -1, 2418, -1, -1, + 2421, -1, 2423, -1, -1, -1, -1, -1, -1, -1, + 2431, -1, -1, -1, -1, -1, -1, -1, 2439, 2440, + -1, -1, -1, -1, -1, 2446, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 1681, -1, -1, -1, -1, -1, -1, 454, -1, + 2461, -1, -1, -1, -1, -1, 382, -1, -1, -1, + 2471, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 380, 3, 2544, -1, 2546, -1, - 8, -1, -1, 11, -1, -1, -1, 15, 16, 17, - 18, 19, 20, 21, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 36, -1, - -1, -1, 40, -1, -1, -1, -1, -1, 46, -1, - -1, -1, -1, -1, -1, 53, -1, 8, -1, -1, - 11, -1, -1, -1, 15, 16, 17, 18, 19, 20, - 21, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 78, -1, -1, -1, -1, 36, -1, -1, -1, 40, - -1, -1, -1, -1, -1, 46, -1, -1, -1, -1, - -1, -1, 53, -1, -1, -1, 2644, 2645, 484, -1, - -1, 487, 488, 489, -1, 491, 492, 493, 494, 495, - 496, -1, -1, 2661, -1, -1, -1, 78, 504, -1, - -1, -1, -1, -1, -1, 2673, -1, -1, -1, -1, - 2678, 2679, -1, -1, -1, 2683, -1, -1, -1, -1, - 2688, -1, -1, 2691, 2692, -1, 1856, -1, 2696, 2697, - -1, -1, 2700, -1, 1864, 1865, -1, 1867, 1868, 1869, - 1870, 1871, 1872, 2711, 172, 1875, 1876, 1877, 1878, 1879, - 1880, 1881, 1882, 1883, 1884, 1885, -1, -1, -1, -1, - 188, -1, -1, -1, -1, 193, -1, -1, -1, -1, - 8, -1, -1, 11, -1, -1, -1, 15, 16, 17, - 18, 19, 20, 21, -1, -1, -1, 2755, 216, 217, - -1, 172, -1, -1, -1, -1, -1, -1, 36, -1, - -1, -1, -1, 231, -1, -1, -1, 188, 46, -1, - -1, -1, 193, -1, 2782, 53, -1, -1, -1, -1, + 2491, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 1963, 1964, 216, 217, -1, -1, 267, - 78, -1, 270, -1, -1, -1, -1, -1, -1, -1, - 231, -1, -1, -1, -1, -1, 284, -1, -1, 287, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 487, + -1, -1, 490, 491, 492, -1, 494, 495, 496, 497, + 498, 499, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 2547, -1, 2549, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 2003, -1, -1, -1, 2007, 2008, 2009, - 2010, 2011, 2012, 2013, 2014, -1, 267, -1, -1, 270, - 2020, 2021, -1, 2023, 2024, -1, -1, -1, -1, 2867, - -1, -1, -1, 284, -1, 2035, 287, -1, 2038, -1, - -1, -1, -1, -1, -1, -1, 2046, 2047, 2048, 2049, - 2050, 2051, 2052, 2053, 2054, 2055, -1, -1, -1, -1, - -1, -1, -1, -1, 172, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 2914, -1, -1, -1, - 188, 2081, 380, -1, -1, 193, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 2934, -1, -1, -1, - -1, -1, -1, 2941, -1, -1, -1, -1, 216, 217, - -1, -1, -1, -1, 2952, 2953, -1, -1, 2956, -1, - 2958, -1, -1, 231, -1, -1, -1, -1, -1, 380, - -1, -1, -1, -1, 2134, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 2983, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 267, - -1, -1, 270, -1, -1, -1, -1, -1, -1, 3007, - -1, -1, -1, -1, -1, -1, 284, -1, -1, 287, - -1, -1, -1, -1, -1, -1, 484, -1, -1, 487, - 488, 489, -1, 491, 492, 493, 494, 495, 496, -1, - -1, -1, 2202, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 2212, 2213, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 484, 3072, -1, 487, 488, 489, -1, - 491, 492, 493, 494, 495, 496, -1, -1, -1, -1, - 501, -1, 3090, -1, -1, -1, 3094, -1, -1, -1, - 3098, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 3108, -1, 380, -1, -1, 3113, -1, 3115, -1, -1, - -1, -1, -1, -1, -1, 3123, -1, -1, -1, -1, - -1, 2291, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 3141, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 2313, 2314, 2315, -1, -1, 2318, 2319, - 2320, 2321, 2322, 2323, -1, -1, -1, 2327, 2328, 2329, - 2330, 2331, 2332, 2333, 2334, 2335, 2336, -1, -1, -1, - -1, 2341, 2342, -1, 8, -1, -1, 11, -1, -1, - 3188, 15, 16, 17, 18, 19, 20, 21, -1, -1, - 3198, -1, -1, -1, -1, -1, -1, -1, -1, 2369, - -1, -1, 36, 3211, -1, 2375, 484, -1, -1, 487, - 488, 489, 46, 491, 492, 493, 494, 495, 496, 53, - -1, 499, -1, 2393, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 3243, 2406, -1, -1, 2409, - 2410, -1, -1, -1, 78, -1, 2416, 2417, -1, -1, + -1, 487, -1, -1, 490, 491, 492, -1, 494, 495, + 496, 497, 498, 499, 1859, -1, -1, -1, -1, -1, + -1, -1, 1867, 1868, -1, 1870, 1871, 1872, 1873, 1874, + 1875, -1, -1, 1878, 1879, 1880, 1881, 1882, 1883, 1884, + 1885, 1886, 1887, 1888, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 2430, 2431, 2432, 2433, -1, 2435, -1, -1, -1, 2439, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 2647, 2648, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 2664, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 2676, -1, -1, -1, -1, + 2681, 2682, -1, -1, -1, 2686, -1, -1, -1, -1, + 2691, 1966, 1967, 2694, 2695, -1, -1, -1, 2699, 2700, + -1, -1, 2703, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 2714, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 26, -1, -1, -1, -1, -1, 32, -1, - -1, -1, -1, -1, -1, -1, 40, -1, -1, 8, - -1, -1, 11, 2493, 3332, 3333, 15, 16, 17, 18, - 19, 20, 21, -1, -1, -1, 60, -1, 172, -1, - -1, -1, -1, -1, -1, -1, -1, 36, -1, -1, - 3358, -1, -1, -1, 188, -1, -1, 46, -1, 193, - -1, -1, -1, -1, 53, -1, -1, -1, -1, -1, + -1, 2006, -1, -1, -1, 2010, 2011, 2012, 2013, 2014, + 2015, 2016, 2017, -1, -1, -1, -1, -1, 2023, 2024, + -1, 2026, 2027, -1, -1, -1, -1, 2758, -1, -1, + -1, -1, -1, 2038, -1, -1, 2041, -1, -1, -1, + -1, -1, -1, -1, 2049, 2050, 2051, 2052, 2053, 2054, + 2055, 2056, 2057, 2058, 2785, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 104, -1, 216, 217, -1, -1, 3394, -1, -1, 78, - -1, -1, -1, -1, -1, -1, -1, 231, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 8, -1, 133, - 11, -1, -1, -1, 15, 16, 17, 18, 19, 20, - 21, -1, -1, 2593, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 267, -1, 36, 270, -1, -1, -1, - -1, -1, -1, -1, -1, 46, -1, -1, -1, -1, - 284, -1, 53, 287, 2624, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 2084, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 196, -1, -1, -1, -1, 78, -1, -1, - -1, -1, -1, 172, -1, -1, -1, -1, -1, -1, - -1, -1, 2662, -1, 2664, -1, -1, -1, 2668, 188, - -1, -1, -1, -1, 193, -1, -1, 2677, -1, -1, - 2680, -1, 2682, -1, -1, -1, 2686, -1, -1, 2689, - 2690, 245, -1, 2693, 2694, -1, -1, 216, 217, 253, - -1, 2701, -1, -1, -1, -1, -1, -1, -1, -1, - 2710, 265, 231, -1, -1, -1, 380, -1, -1, -1, - -1, -1, -1, -1, -1, 2725, -1, -1, -1, -1, - -1, -1, 286, -1, -1, -1, -1, -1, -1, -1, - -1, 172, 296, -1, -1, -1, -1, -1, 267, -1, - -1, 270, -1, -1, -1, -1, -1, 188, -1, -1, - -1, -1, 193, 2763, -1, 284, -1, -1, 287, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 216, 217, 341, -1, -1, - -1, 345, -1, 347, -1, -1, -1, -1, -1, -1, - 231, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 370, -1, -1, -1, - 484, 375, -1, 487, 488, 489, -1, 491, 492, 493, - 494, 495, 496, -1, -1, 389, 267, 501, -1, 270, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 284, -1, -1, 287, -1, -1, -1, - -1, 380, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 445, -1, -1, -1, -1, -1, 2897, 2898, -1, + -1, -1, 2137, -1, -1, -1, -1, -1, -1, 2870, + -1, -1, -1, 0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 22, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 32, -1, 34, 35, -1, + -1, -1, -1, -1, -1, -1, 2917, -1, -1, -1, + 47, -1, -1, -1, -1, 52, -1, -1, -1, -1, + 2205, -1, -1, -1, 61, -1, 2937, -1, -1, -1, + 2215, 2216, -1, 2944, -1, -1, 73, -1, -1, -1, + -1, -1, -1, -1, 2955, 2956, -1, 84, 2959, -1, + 2961, -1, -1, -1, -1, -1, -1, -1, -1, 96, + -1, 98, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 109, -1, -1, 2986, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 123, 124, -1, -1, + -1, -1, -1, -1, -1, -1, 133, -1, -1, 3010, + -1, -1, 139, -1, -1, -1, -1, -1, -1, 2294, + 147, -1, 149, 150, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 163, -1, -1, -1, + 167, 2316, 2317, 2318, -1, -1, 2321, 2322, 2323, 2324, + 2325, 2326, -1, -1, -1, 2330, 2331, 2332, 2333, 2334, + 2335, 2336, 2337, 2338, 2339, -1, 193, -1, -1, 2344, + 2345, -1, -1, -1, 3075, -1, -1, -1, -1, -1, + -1, -1, 209, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 3093, -1, -1, -1, 3097, 2372, -1, -1, + 3101, -1, -1, 2378, -1, -1, -1, -1, 235, -1, + 3111, -1, -1, -1, -1, 3116, -1, 3118, -1, -1, + -1, 2396, -1, -1, -1, 3126, -1, -1, -1, -1, + -1, -1, -1, -1, 2409, -1, -1, 2412, 2413, -1, + -1, -1, -1, 3144, 2419, 2420, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 2433, 2434, + 2435, 2436, -1, 2438, -1, -1, -1, 2442, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 2922, 2923, -1, -1, -1, -1, -1, -1, + 307, 308, 309, -1, -1, -1, -1, -1, 315, -1, + 3191, 318, -1, -1, -1, -1, -1, -1, -1, -1, + 3201, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 3214, -1, -1, -1, -1, -1, -1, + 347, 2496, -1, 8, -1, -1, 11, -1, -1, 356, + 15, 16, 17, 18, 19, 20, 21, -1, -1, -1, + -1, -1, -1, 370, -1, 3246, -1, -1, -1, -1, + 377, 36, -1, -1, 381, 40, -1, -1, -1, -1, + -1, 46, -1, -1, 391, -1, -1, -1, 53, -1, + -1, -1, -1, -1, -1, -1, 403, -1, -1, -1, + 407, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 78, -1, 8, -1, -1, 11, -1, + 427, -1, 15, 16, 17, 18, 19, 20, 21, -1, + -1, -1, -1, 440, -1, -1, 443, -1, -1, 446, + -1, 2596, -1, 36, 451, -1, -1, -1, -1, -1, + -1, -1, -1, 46, 3335, 3336, 3337, -1, 465, -1, + 53, -1, -1, 8, -1, -1, 11, -1, -1, -1, + 15, 16, 2627, -1, 19, 20, 21, -1, -1, -1, + -1, 3362, 489, -1, -1, 78, -1, -1, -1, -1, + -1, 36, -1, -1, -1, -1, 503, -1, -1, 506, + -1, 46, -1, -1, -1, -1, -1, 172, 53, -1, + 2665, -1, 2667, -1, -1, -1, 2671, -1, -1, -1, + -1, -1, 3403, -1, 189, 2680, -1, -1, 2683, 194, + 2685, -1, -1, 78, 2689, -1, -1, 2692, 2693, -1, + -1, 2696, 2697, -1, -1, -1, -1, -1, -1, 2704, + -1, -1, 217, 218, -1, -1, -1, -1, 2713, -1, + -1, -1, -1, -1, -1, -1, -1, 232, -1, -1, + -1, -1, -1, 2728, -1, -1, -1, -1, -1, 172, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 2940, -1, -1, -1, 2944, -1, 2946, 2947, 2948, 380, - -1, 2951, -1, -1, 2954, 2955, -1, -1, -1, -1, - -1, -1, -1, 2963, -1, 484, -1, -1, 487, 488, - 489, -1, 491, 492, 493, 494, 495, 496, -1, -1, - -1, -1, 501, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 189, -1, -1, -1, + -1, 194, -1, 268, -1, -1, 271, -1, -1, -1, + -1, 2766, -1, -1, -1, -1, -1, -1, -1, -1, + 285, -1, -1, 288, 217, 218, -1, 172, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 232, + -1, -1, -1, -1, 189, -1, -1, -1, -1, 194, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3004, -1, -1, -1, -1, -1, - 3010, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 3023, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 217, 218, -1, 268, -1, -1, 271, -1, + -1, -1, -1, -1, -1, -1, -1, 232, -1, -1, + -1, -1, 285, -1, -1, 288, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 3052, 484, -1, -1, 487, 488, 489, -1, - 491, 492, 493, 494, 495, 496, -1, -1, -1, -1, - 501, -1, -1, -1, -1, -1, -1, -1, 3078, -1, + -1, -1, -1, -1, -1, -1, -1, 382, -1, -1, + -1, -1, -1, 268, -1, -1, 271, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 285, -1, -1, -1, -1, 2900, 2901, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 3110, 3111, 3112, -1, -1, -1, -1, -1, -1, -1, + 2925, 2926, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 2943, 382, + -1, -1, 2947, -1, 2949, 2950, 2951, -1, -1, 2954, + -1, -1, 2957, 2958, -1, -1, -1, -1, -1, -1, + -1, 2966, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 487, -1, -1, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, -1, -1, 382, -1, 504, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 3130, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 3142, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 3007, -1, -1, -1, -1, -1, 3013, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 3026, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 487, -1, -1, 490, 491, 492, + 3055, 494, 495, 496, 497, 498, 499, -1, -1, -1, + -1, 504, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 3081, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 3186, -1, -1, -1, + -1, -1, 487, -1, -1, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, -1, -1, -1, 3113, 3114, + 3115, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 3133, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 3203, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 3213, -1, -1, -1, -1, -1, -1, + 3145, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 3238, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 3263, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 3, 4, 5, 6, 7, 8, 9, - 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, 3307, -1, 39, - -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, - 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, - 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, - 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, - -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, - 90, 3361, 92, 93, 94, 95, 96, 97, -1, 99, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, -1, 116, 117, 118, 119, - 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, - 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, - 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, - 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, - 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, 188, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, 203, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, 216, 217, 218, 219, - 220, 221, 222, 223, 224, 225, -1, -1, 228, 229, - 230, 231, -1, 233, 234, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, 312, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 382, 383, 384, -1, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, 407, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, 452, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, - 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, -1, -1, -1, -1, -1, - -1, 491, 492, 493, -1, -1, -1, -1, 498, -1, - 500, -1, -1, -1, -1, 505, 506, 507, 508, -1, - 510, 3, 4, 5, 6, 7, 8, 9, 10, -1, + -1, -1, -1, -1, 3189, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, - 32, 33, -1, -1, -1, -1, -1, 39, -1, -1, - 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, - 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, - 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, - -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, - 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, - 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, -1, 116, 117, 118, 119, 120, 121, - -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, - 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, - 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, - -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, - 172, 173, -1, 175, -1, 177, 178, 179, -1, 181, - 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, -1, -1, 228, 229, 230, 231, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - 312, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, -1, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, 407, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - 452, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, - 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, -1, -1, -1, -1, -1, -1, 491, - 492, 493, -1, -1, -1, -1, 498, -1, 500, -1, - -1, -1, -1, 505, 506, 507, 508, -1, 510, 3, + -1, 3206, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 3216, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 3, 4, + 5, 6, 7, 8, 9, 10, 3241, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, + 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, + -1, 3266, -1, -1, 39, -1, -1, 42, 43, 44, + -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, + 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, + 65, 66, 67, 68, 69, 70, 71, -1, 73, 74, + 75, 76, 77, -1, 79, 3310, 81, 82, 83, 84, + 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, + 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, + 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, + 3365, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, + 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, + 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, 204, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 226, -1, -1, 229, 230, 231, 232, -1, 234, + 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, + 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, + 275, 276, 277, 278, 279, -1, 281, 282, -1, -1, + 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, -1, -1, -1, -1, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, + 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, 409, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + 455, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, -1, -1, -1, -1, -1, -1, 494, + 495, 496, -1, -1, -1, -1, 501, -1, 503, -1, + -1, -1, -1, 508, 509, 510, 511, -1, 513, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, @@ -12029,247 +12058,96 @@ static const yytype_int16 yycheck[] = 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, - -1, 165, 166, 167, 168, 169, 170, 171, 172, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, 188, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, 203, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, 216, 217, 218, 219, 220, 221, 222, 223, - 224, 225, -1, -1, 228, 229, 230, 231, -1, 233, + -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, -1, - 284, 285, 286, -1, -1, 289, 290, 291, 292, 293, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, -1, 309, 310, 311, 312, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, 352, 353, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, 368, -1, 370, 371, 372, 373, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, 405, 406, 407, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, 437, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, 452, 453, - 454, 455, 456, 457, -1, -1, 460, -1, 462, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, -1, 486, -1, -1, -1, -1, 491, 492, 493, - -1, -1, -1, -1, 498, -1, 500, 501, -1, -1, - -1, 505, 506, 507, 508, 3, 4, 5, 6, 7, - 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, - 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, - -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, - 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, - -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, - 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, - -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, - 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, - -1, 99, 100, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, 113, 114, -1, 116, 117, - 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, - -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, - 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, - 158, 159, 160, 161, -1, 163, -1, 165, 166, 167, - 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, 203, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, 216, 217, - 218, 219, 220, 221, 222, 223, 224, 225, -1, -1, - 228, 229, 230, 231, -1, 233, 234, 235, 236, 237, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - -1, 309, 310, 311, 312, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, 352, 353, 354, 355, 356, 357, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - 368, -1, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 382, 383, 384, -1, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, 405, 406, 407, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, - 428, 429, 430, 431, -1, -1, 434, 435, 436, 437, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, 452, 453, 454, 455, 456, 457, - -1, -1, 460, -1, 462, 463, 464, 465, 466, 467, - 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 484, -1, 486, -1, - -1, -1, -1, 491, 492, 493, -1, -1, -1, -1, - 498, -1, 500, -1, -1, -1, -1, 505, 506, 507, - 508, 3, 4, 5, 6, 7, 8, 9, 10, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, - 32, 33, -1, -1, -1, 37, -1, 39, -1, -1, - 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, - 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, - 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, - -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, - 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, - 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, -1, 116, 117, 118, 119, 120, 121, - -1, 123, 124, 125, 126, 127, 128, -1, 130, 131, - 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, - 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, - -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, - 172, 173, -1, 175, -1, 177, 178, 179, -1, 181, - 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, -1, -1, 228, 229, 230, 231, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - 312, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, 407, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, 450, 451, - 452, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, - 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, -1, -1, -1, -1, -1, -1, 491, - 492, 493, -1, -1, -1, -1, 498, -1, 500, 501, - -1, -1, -1, 505, 506, 507, 508, 3, 4, 5, - 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, - 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, - -1, -1, -1, 39, -1, -1, 42, 43, 44, -1, - 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, - 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, - 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, - 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, - 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, - 96, 97, -1, 99, 100, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, -1, - 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, - 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, - 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, - 166, 167, 168, 169, 170, 171, 172, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, 203, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, - 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, - -1, 227, 228, 229, 230, 231, -1, 233, 234, 235, - 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, -1, 284, 285, - 286, -1, -1, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, 312, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, 352, 353, 354, 355, - 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, -1, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, 407, 408, 409, 410, 411, 412, -1, 414, 415, - -1, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, 452, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, - 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, -1, - -1, -1, -1, -1, -1, 491, 492, 493, -1, -1, - -1, -1, 498, -1, 500, -1, -1, -1, -1, 505, - 506, 507, 508, 3, 4, 5, 6, 7, 8, 9, - 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, 37, -1, 39, - -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, - 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, - 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, - 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, - -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, - 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, -1, 116, 117, 118, 119, - 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, - 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, - 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, - 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, - 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, 188, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, 203, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, 216, 217, 218, 219, - 220, 221, 222, 223, 224, 225, -1, -1, 228, 229, - 230, 231, -1, 233, 234, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, 312, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, 369, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 382, 383, 384, -1, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, 407, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - 450, 451, 452, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, - 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, -1, -1, -1, -1, -1, - -1, 491, 492, 493, -1, -1, -1, -1, 498, -1, - 500, -1, -1, -1, -1, 505, 506, 507, 508, 3, + 484, 485, 486, 487, -1, -1, -1, -1, -1, -1, + 494, 495, 496, -1, -1, -1, -1, 501, -1, 503, + -1, -1, -1, -1, 508, 509, 510, 511, -1, 513, + 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, + 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, + 33, -1, -1, -1, -1, -1, 39, -1, -1, 42, + 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, + 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, + 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, + 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, + 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, + 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, -1, 116, 117, 118, 119, 120, 121, -1, + 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, + 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, + 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, + 163, -1, 165, 166, 167, 168, 169, 170, 171, 172, + 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, + 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, 204, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, 217, 218, 219, 220, 221, 222, + 223, 224, 225, 226, -1, -1, 229, 230, 231, 232, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, + 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, + 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + 273, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, + 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 308, 309, -1, 311, 312, + 313, 314, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, + 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, 370, -1, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, 440, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, 455, 456, 457, 458, 459, 460, -1, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, 472, + 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, -1, 489, -1, -1, -1, + -1, 494, 495, 496, -1, -1, -1, -1, 501, -1, + 503, 504, -1, -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, - -1, -1, -1, 37, -1, 39, -1, -1, 42, 43, + -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, @@ -12282,148 +12160,98 @@ static const yytype_int16 yycheck[] = 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, - -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, 188, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, 203, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, 216, 217, 218, 219, 220, 221, 222, 223, - 224, 225, -1, -1, 228, 229, 230, 231, -1, 233, + -1, 165, 166, 167, 168, 169, 170, 171, 172, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, -1, - 284, 285, 286, -1, -1, 289, 290, 291, 292, 293, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, 312, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, 352, 353, + 304, 305, 306, 307, 308, 309, -1, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, 369, 370, 371, 372, 373, + 364, 365, 366, 367, 368, 369, 370, -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, 407, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, 452, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, 440, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, -1, -1, -1, -1, -1, -1, 491, 492, 493, - -1, -1, -1, -1, 498, -1, 500, 501, -1, -1, - -1, 505, 506, 507, 508, 3, 4, 5, 6, 7, - 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, - 28, 29, 30, 31, 32, 33, -1, -1, -1, -1, - 38, 39, -1, -1, 42, 43, 44, -1, 46, 47, - 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, - -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, - 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, - -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, - 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, - -1, 99, 100, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, 113, 114, -1, 116, 117, - 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, - -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, - 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, - 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, - 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, 203, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, 216, 217, - 218, 219, 220, 221, 222, 223, 224, 225, -1, -1, - 228, 229, 230, 231, -1, 233, 234, 235, 236, 237, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, -1, -1, -1, - -1, 309, 310, 311, 312, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, 352, 353, 354, 355, 356, 357, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - -1, -1, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, -1, 406, 407, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, - 428, 429, 430, 431, -1, -1, 434, 435, 436, -1, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, 452, 453, 454, 455, 456, 457, - -1, -1, 460, -1, -1, 463, 464, 465, 466, 467, - 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 484, -1, -1, -1, - -1, -1, -1, 491, 492, 493, -1, -1, -1, -1, - 498, -1, 500, -1, -1, -1, -1, 505, 506, 507, - 508, 3, 4, 5, 6, 7, 8, 9, 10, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, - 32, 33, -1, -1, -1, 37, -1, 39, -1, -1, - 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, - 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, - 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, - -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, - 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, - 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, -1, 116, 117, 118, 119, 120, 121, - -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, - 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, - 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, - -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, - 172, 173, -1, 175, -1, 177, 178, 179, -1, 181, - 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, -1, -1, 228, 229, 230, 231, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - 312, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, 407, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - 452, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, - 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, -1, -1, -1, -1, -1, -1, 491, - 492, 493, -1, -1, -1, -1, 498, -1, 500, 501, - -1, -1, -1, 505, 506, 507, 508, 3, 4, 5, + 484, 485, 486, 487, -1, 489, -1, -1, -1, -1, + 494, 495, 496, -1, -1, -1, -1, 501, -1, 503, + -1, -1, -1, -1, 508, 509, 510, 511, 3, 4, + 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, + 25, 26, 27, 28, 29, 30, 31, 32, 33, -1, + -1, -1, 37, -1, 39, -1, -1, 42, 43, 44, + -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, + 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, + 65, 66, 67, 68, 69, 70, 71, -1, 73, 74, + 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, + 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, + 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, + 125, 126, 127, 128, -1, 130, 131, 132, 133, 134, + -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, + 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, + 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, 204, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 226, -1, -1, 229, 230, 231, 232, -1, 234, + 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, + 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, + 275, 276, 277, 278, 279, -1, 281, 282, -1, -1, + 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, -1, -1, -1, -1, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, + 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, -1, 371, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, 409, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, 453, 454, + 455, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, -1, -1, -1, -1, -1, -1, 494, + 495, 496, -1, -1, -1, -1, 501, -1, 503, 504, + -1, -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, - -1, 37, -1, 39, -1, -1, 42, 43, 44, -1, + -1, -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, - 56, 57, -1, 59, 60, 61, 62, 63, 64, 65, + 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, @@ -12434,144 +12262,45 @@ static const yytype_int16 yycheck[] = 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, - 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, 203, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, + 166, 167, 168, 169, 170, 171, 172, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, 204, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, - -1, -1, 228, 229, 230, 231, -1, 233, 234, 235, + 226, -1, 228, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, -1, 284, 285, - 286, -1, -1, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, 312, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, 352, 353, 354, 355, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, 407, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, 452, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, 409, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, 455, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, -1, - -1, -1, -1, -1, -1, 491, 492, 493, -1, -1, - -1, -1, 498, -1, 500, -1, -1, -1, -1, 505, - 506, 507, 508, 3, 4, 5, 6, 7, -1, 9, - 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, -1, -1, 39, - -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, - 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, - 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, - 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, - -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, - 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, - 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, - 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, - 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, - 160, 161, -1, 163, -1, 165, 166, 167, 168, 169, - 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, 188, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, 203, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, 216, 217, 218, 219, - 220, 221, 222, 223, 224, 225, -1, -1, 228, 229, - 230, 231, -1, 233, 234, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, 261, 262, 263, 264, 265, 266, -1, 268, 269, - 270, 271, 272, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, -1, 309, - 310, 311, 312, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, 368, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 382, 383, 384, -1, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, 405, 406, 407, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, 437, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, 452, 453, 454, 455, 456, 457, -1, -1, - 460, -1, 462, 463, 464, 465, 466, 467, 468, 469, - 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, -1, -1, 486, -1, -1, -1, - 3, 4, 5, 6, 7, 8, 9, 10, 498, -1, - 500, -1, -1, -1, -1, 505, 506, 507, 508, 22, - 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, - 33, -1, -1, -1, -1, -1, 39, -1, -1, 42, - 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, - 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, - 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, - 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - 113, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - 203, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, 216, 217, 218, 219, 220, 221, 222, - 223, 224, 225, -1, -1, 228, 229, 230, 231, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, - 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, 312, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, 352, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, 407, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, 452, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 484, -1, -1, -1, -1, -1, -1, 491, 492, - 493, -1, -1, -1, -1, 498, -1, 500, -1, -1, - -1, -1, 505, 506, 507, 508, 3, 4, 5, 6, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, -1, -1, -1, -1, -1, -1, 494, 495, + 496, -1, -1, -1, -1, 501, -1, 503, -1, -1, + -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, - 27, 28, 29, 30, 31, 32, 33, -1, -1, -1, - -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, + 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, + 37, -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, @@ -12585,144 +12314,94 @@ static const yytype_int16 yycheck[] = 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, 203, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, 216, - 217, 218, 219, 220, 221, 222, 223, 224, 225, -1, - -1, 228, 229, 230, 231, -1, 233, 234, 235, 236, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, 189, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, 204, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, + -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, 312, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, 352, 353, 354, 355, 356, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, 314, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - 407, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, 452, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, + 367, 368, 369, -1, 371, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, 409, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, 426, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, + 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, -1, -1, - -1, -1, -1, -1, 491, 492, 493, -1, -1, -1, - -1, 498, -1, 500, -1, -1, -1, -1, 505, 506, - 507, 508, 3, 4, 5, 6, 7, 8, 9, 10, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, -1, -1, 39, -1, - -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, - 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, - 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, - 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, - 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, 166, 167, 168, 169, 170, - 171, 172, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, 203, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, 216, 217, 218, 219, 220, - 221, 222, 223, 224, 225, -1, -1, 228, 229, 230, - 231, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, 312, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, -1, 406, 407, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, 452, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, - 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, -1, -1, -1, -1, -1, -1, - 491, 492, 493, -1, -1, -1, -1, 498, -1, 500, - -1, -1, -1, -1, 505, 506, 507, 508, 3, 4, - 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, -1, -1, 39, -1, -1, 42, 43, 44, - -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, - 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, - 65, 66, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, - 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, 203, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, - 225, -1, -1, 228, 229, 230, 231, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, - 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, 312, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, 352, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, 407, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, 452, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - -1, -1, -1, -1, -1, -1, 491, 492, 493, -1, - -1, -1, -1, 498, -1, 500, 501, -1, -1, -1, - 505, 506, 507, 508, 3, 4, 5, 6, 7, 8, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 487, -1, -1, -1, -1, -1, -1, 494, 495, 496, + -1, -1, -1, -1, 501, -1, 503, -1, -1, -1, + -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, + 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, + 28, 29, -1, 31, 32, 33, -1, -1, -1, 37, + -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, + 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, + -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, + 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, + -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, + 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, + -1, 99, 100, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, -1, 116, 117, + 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, + -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, + 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, + 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, + 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, 204, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, 217, + 218, 219, 220, 221, 222, 223, 224, 225, 226, -1, + -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, + 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, + 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, -1, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 384, 385, 386, -1, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, 409, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, 426, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, 455, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + -1, -1, -1, -1, -1, -1, 494, 495, 496, -1, + -1, -1, -1, 501, -1, 503, 504, -1, -1, -1, + 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, - 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, + 29, 30, 31, 32, 33, -1, -1, -1, -1, 38, 39, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, @@ -12737,42 +12416,244 @@ static const yytype_int16 yycheck[] = 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, 203, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, 216, 217, 218, - 219, 220, 221, 222, 223, 224, 225, -1, -1, 228, - 229, 230, 231, -1, 233, 234, 235, 236, 237, 238, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, 204, 205, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, -1, -1, + 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, + 279, -1, 281, 282, -1, -1, 285, 286, 287, -1, + -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, 352, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 382, 383, 384, -1, 386, 387, 388, + 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, 405, 406, 407, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, 452, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, + 399, 400, 401, 402, 403, -1, 405, 406, -1, 408, + 409, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, 423, 424, 425, 426, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, 451, 452, -1, 454, 455, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, -1, -1, -1, -1, - -1, -1, 491, 492, 493, -1, -1, -1, -1, 498, - -1, 500, -1, -1, -1, -1, 505, 506, 507, 508, + 479, 480, 481, 482, 483, 484, 485, 486, 487, -1, + -1, -1, -1, -1, -1, 494, 495, 496, -1, -1, + -1, -1, 501, -1, 503, -1, -1, -1, -1, 508, + 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, + 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, -1, -1, -1, 37, -1, 39, + -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, + 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, + 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, + -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, 113, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, + 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, -1, -1, 229, + 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, -1, 405, 406, -1, 408, 409, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, 426, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, 455, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 487, -1, -1, + -1, -1, -1, -1, 494, 495, 496, -1, -1, -1, + -1, 501, -1, 503, 504, -1, -1, -1, 508, 509, + 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, + 31, 32, 33, -1, -1, -1, 37, -1, 39, -1, + -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, + 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, + 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, + 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, + 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, + -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, + 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, -1, 116, 117, 118, 119, 120, + 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, + 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, + 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, + 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, + 171, 172, 173, -1, 175, -1, 177, 178, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, 204, 205, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, -1, -1, 229, 230, + 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, + 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 274, 275, 276, 277, 278, 279, -1, + 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 402, 403, -1, 405, 406, -1, 408, 409, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 431, 432, 433, 434, -1, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + 451, 452, -1, 454, 455, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, + 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 487, -1, -1, -1, + -1, -1, -1, 494, 495, 496, -1, -1, -1, -1, + 501, -1, 503, -1, -1, -1, -1, 508, 509, 510, + 511, 3, 4, 5, 6, 7, -1, 9, 10, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, + 32, 33, -1, -1, -1, -1, -1, 39, -1, -1, + 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, + 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, + 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, + -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, + 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, + 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, + -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, + 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, + -1, 163, -1, 165, 166, 167, 168, 169, 170, 171, + 172, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, 204, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, -1, -1, 229, 230, 231, + 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 265, 266, 267, -1, 269, 270, 271, + 272, 273, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 309, -1, 311, + 312, 313, 314, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, 370, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 402, 403, -1, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, 440, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, 455, 456, 457, 458, 459, 460, -1, + -1, 463, -1, 465, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, -1, -1, 489, -1, -1, + -1, 3, 4, 5, 6, 7, 8, 9, 10, 501, + -1, 503, -1, -1, -1, -1, 508, 509, 510, 511, + 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, + 32, 33, -1, -1, -1, -1, -1, 39, -1, -1, + 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, + 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, + 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, + -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, + 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, + 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, 113, 114, -1, 116, 117, 118, 119, 120, 121, + -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, + 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, + -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, + 172, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, 204, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, -1, -1, 229, 230, 231, + 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, 314, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 402, 403, -1, 405, 406, -1, 408, 409, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, 455, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 487, -1, -1, -1, -1, + -1, -1, 494, 495, 496, -1, -1, -1, -1, 501, + -1, 503, -1, -1, -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, + 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, @@ -12787,141 +12668,91 @@ static const yytype_int16 yycheck[] = -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, + 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - 203, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, 216, 217, 218, 219, 220, 221, 222, - 223, 224, 225, -1, -1, 228, 229, 230, 231, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, 204, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, 217, 218, 219, 220, 221, 222, + 223, 224, 225, 226, -1, -1, 229, 230, 231, 232, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, 291, 292, + 273, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, 312, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, 352, + 303, 304, 305, 306, -1, -1, -1, -1, 311, 312, + 313, 314, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, + 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, 407, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, 452, - 453, 454, 455, 456, 457, -1, 459, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, -1, 408, 409, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, -1, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, 455, 456, 457, 458, 459, 460, -1, -1, + 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 484, -1, -1, -1, -1, -1, -1, 491, 492, - 493, -1, -1, -1, -1, 498, -1, 500, -1, -1, - -1, -1, 505, 506, 507, 508, 3, 4, 5, 6, - 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, - 27, 28, 29, 30, 31, 32, 33, -1, -1, -1, - -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, - 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, - 57, -1, 59, 60, 61, 62, 63, 64, 65, 66, - 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, - 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, - 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, - 97, -1, 99, 100, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, -1, 116, - 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, - 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, - -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, - -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, 203, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, 216, - 217, 218, 219, 220, 221, 222, 223, 224, 225, -1, - -1, 228, 229, 230, 231, -1, 233, 234, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, - 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, 312, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, 352, 353, 354, 355, 356, - 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - 407, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, 452, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, - 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, -1, -1, - -1, -1, -1, -1, 491, 492, 493, -1, -1, -1, - -1, 498, -1, 500, -1, -1, -1, -1, 505, 506, - 507, 508, 3, 4, 5, 6, 7, 8, 9, 10, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, -1, -1, 39, -1, - -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, - 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, - 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, - 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, - 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, - 171, 172, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, 203, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, 216, 217, 218, 219, 220, - 221, 222, 223, 224, 225, -1, -1, 228, 229, 230, - 231, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, 312, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, 405, 406, 407, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, 452, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, - 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, -1, -1, -1, -1, -1, -1, - 491, 492, 493, -1, -1, -1, -1, 498, -1, 500, - -1, -1, -1, -1, 505, 506, 507, 508, 3, 4, + 483, 484, 485, 486, 487, -1, -1, -1, -1, -1, + -1, 494, 495, 496, -1, -1, -1, -1, 501, -1, + 503, -1, -1, -1, -1, 508, 509, 510, 511, 3, + 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, + 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, + 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, + 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, + 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, 166, 167, 168, 169, 170, 171, 172, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, 232, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, + 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, 487, -1, -1, -1, -1, -1, -1, + 494, 495, 496, -1, -1, -1, -1, 501, -1, 503, + -1, -1, -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, @@ -12939,141 +12770,91 @@ static const yytype_int16 yycheck[] = 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, 203, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, - 225, -1, -1, 228, 229, 230, 231, -1, 233, 234, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, 204, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, 291, 292, 293, 294, + 275, 276, 277, 278, 279, -1, 281, 282, -1, -1, + 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, 312, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, 352, 353, 354, + 305, 306, -1, -1, -1, -1, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, + 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - 405, 406, 407, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, 452, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, 409, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + 455, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - -1, -1, -1, -1, -1, -1, 491, 492, 493, -1, - -1, -1, -1, 498, -1, 500, -1, -1, -1, -1, - 505, 506, 507, 508, 3, 4, 5, 6, 7, 8, - 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, - 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, - 39, -1, -1, 42, 43, 44, -1, 46, 47, 48, - 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, - 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, - 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, - 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, - 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, - 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, -1, 116, 117, 118, - 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, - -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, - 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, - 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, - 169, 170, 171, 172, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, 203, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, 216, 217, 218, - 219, 220, 221, 222, 223, 224, 225, -1, -1, 228, - 229, 230, 231, -1, 233, 234, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, - 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, 352, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 382, 383, 384, -1, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, -1, 406, 407, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, 452, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, - 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, -1, -1, -1, -1, - -1, -1, 491, 492, 493, -1, -1, -1, -1, 498, - -1, 500, -1, -1, -1, -1, 505, 506, 507, 508, - 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, - 33, -1, -1, -1, -1, -1, 39, -1, -1, 42, - 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, - 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, - 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, - 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - 113, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - 203, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, 216, 217, 218, 219, 220, 221, 222, - 223, 224, 225, -1, -1, 228, 229, 230, 231, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, - 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, 312, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, 352, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, 407, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, 452, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 484, -1, -1, -1, -1, -1, -1, 491, 492, - 493, -1, -1, -1, -1, 498, -1, 500, -1, -1, - -1, -1, 505, 506, 507, 508, 3, 4, 5, 6, + 485, 486, 487, -1, -1, -1, -1, -1, -1, 494, + 495, 496, -1, -1, -1, -1, 501, -1, 503, 504, + -1, -1, -1, 508, 509, 510, 511, 3, 4, 5, + 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, + 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, + -1, -1, -1, 39, -1, -1, 42, 43, 44, -1, + 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, + 56, 57, -1, 59, 60, 61, 62, 63, 64, 65, + 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, + 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, + 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, + 96, 97, -1, 99, 100, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, -1, + 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, + 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, + 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, + 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, 204, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, + 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + 426, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, 455, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, + 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, -1, -1, -1, -1, -1, -1, 494, 495, + 496, -1, -1, -1, -1, 501, -1, 503, -1, -1, + -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, @@ -13091,141 +12872,91 @@ static const yytype_int16 yycheck[] = 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, 203, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, 216, - 217, 218, 219, 220, 221, 222, 223, 224, 225, -1, - -1, 228, 229, 230, 231, -1, 233, 234, 235, 236, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, 189, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, 204, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, + -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, 312, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, 352, 353, 354, 355, 356, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, 314, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - 407, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, 452, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, + 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, 409, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, 426, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, -1, 454, 455, 456, + 457, 458, 459, 460, -1, 462, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, -1, -1, - -1, -1, -1, -1, 491, 492, 493, -1, -1, -1, - -1, 498, -1, 500, -1, -1, -1, -1, 505, 506, - 507, 508, 3, 4, 5, 6, 7, 8, 9, 10, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, -1, -1, 39, -1, - -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, - 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, - 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, - 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, - 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, - 171, 172, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, 203, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, 216, 217, 218, 219, 220, - 221, 222, 223, 224, 225, -1, -1, 228, 229, 230, - 231, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, 312, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, -1, 406, 407, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, 452, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, - 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, -1, -1, -1, -1, -1, -1, - 491, 492, 493, -1, -1, -1, -1, 498, -1, 500, - -1, -1, -1, -1, 505, 506, 507, 508, 3, 4, - 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, -1, -1, 39, -1, -1, 42, 43, 44, - -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, - 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, - 65, 66, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, - 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, 203, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, - 225, -1, -1, 228, 229, 230, 231, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, - 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, 312, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, 352, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, 407, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, 452, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - -1, -1, -1, -1, -1, -1, 491, 492, 493, -1, - -1, -1, -1, 498, -1, 500, -1, -1, -1, -1, - 505, 506, 507, 508, 3, 4, 5, 6, 7, 8, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 487, -1, -1, -1, -1, -1, -1, 494, 495, 496, + -1, -1, -1, -1, 501, -1, 503, -1, -1, -1, + -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, + 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, + 28, 29, 30, 31, 32, 33, -1, -1, -1, -1, + -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, + 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, + -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, + 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, + -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, + 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, + -1, 99, 100, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, -1, 116, 117, + 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, + -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, + 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, + 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, + 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, 204, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, 217, + 218, 219, 220, 221, 222, 223, 224, 225, 226, -1, + -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, + 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, + 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 384, 385, 386, -1, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, 409, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, 426, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, 455, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + -1, -1, -1, -1, -1, -1, 494, 495, 496, -1, + -1, -1, -1, 501, -1, 503, -1, -1, -1, -1, + 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, @@ -13243,39 +12974,192 @@ static const yytype_int16 yycheck[] = 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, 203, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, 216, 217, 218, - 219, 220, 221, 222, 223, 224, 225, -1, -1, 228, - 229, 230, 231, -1, 233, 234, 235, 236, 237, 238, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, 204, 205, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, -1, -1, + 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, + 279, -1, 281, 282, -1, -1, 285, 286, 287, -1, + -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, 352, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 382, 383, 384, -1, 386, 387, 388, + 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, -1, 406, 407, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, 452, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, + 399, 400, 401, 402, 403, -1, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, 423, 424, 425, 426, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, 451, 452, -1, 454, 455, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, -1, -1, -1, -1, - -1, -1, 491, 492, 493, -1, -1, -1, -1, 498, - -1, 500, -1, -1, -1, -1, 505, 506, 507, 508, + 479, 480, 481, 482, 483, 484, 485, 486, 487, -1, + -1, -1, -1, -1, -1, 494, 495, 496, -1, -1, + -1, -1, 501, -1, 503, -1, -1, -1, -1, 508, + 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, + 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, -1, -1, -1, -1, -1, 39, + -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, + 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, + 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, + -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, 113, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, + 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, -1, -1, 229, + 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, -1, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, 426, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, 455, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 487, -1, -1, + -1, -1, -1, -1, 494, 495, 496, -1, -1, -1, + -1, 501, -1, 503, -1, -1, -1, -1, 508, 509, + 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, + 31, 32, 33, -1, -1, -1, -1, -1, 39, -1, + -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, + 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, + 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, + 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, + 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, + -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, + 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, -1, 116, 117, 118, 119, 120, + 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, + 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, + 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, + 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, + 171, 172, 173, -1, 175, -1, 177, 178, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, 204, 205, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, -1, -1, 229, 230, + 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, + 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 274, 275, 276, 277, 278, 279, -1, + 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 402, 403, -1, 405, 406, -1, 408, 409, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 431, 432, 433, 434, -1, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + 451, 452, -1, 454, 455, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, + 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 487, -1, -1, -1, + -1, -1, -1, 494, 495, 496, -1, -1, -1, -1, + 501, -1, 503, -1, -1, -1, -1, 508, 509, 510, + 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, + 32, 33, -1, -1, -1, -1, -1, 39, -1, -1, + 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, + 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, + 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, + -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, + 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, + 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, 113, 114, -1, 116, 117, 118, 119, 120, 121, + -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, + 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, + -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, + 172, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, 204, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, -1, -1, 229, 230, 231, + 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, 314, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 402, 403, -1, 405, 406, -1, 408, 409, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, 455, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 487, -1, -1, -1, -1, + -1, -1, 494, 495, 496, -1, -1, -1, -1, 501, + -1, 503, -1, -1, -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, @@ -13293,141 +13177,91 @@ static const yytype_int16 yycheck[] = -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, + 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - 203, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, 216, 217, 218, 219, 220, 221, 222, - 223, 224, 225, -1, -1, 228, 229, 230, 231, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, 204, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, 217, 218, 219, 220, 221, 222, + 223, 224, 225, 226, -1, -1, 229, 230, 231, 232, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, 291, 292, + 273, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, 312, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, 352, + 303, 304, 305, 306, -1, -1, -1, -1, 311, 312, + 313, 314, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, + 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, 407, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, 452, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, -1, 408, 409, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, -1, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, 455, 456, 457, 458, 459, 460, -1, -1, + 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 484, -1, -1, -1, -1, -1, -1, 491, 492, - 493, -1, -1, -1, -1, 498, -1, 500, -1, -1, - -1, -1, 505, 506, 507, 508, 3, 4, 5, 6, - 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, - 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, - -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, - 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, - 57, -1, 59, 60, 61, 62, 63, 64, 65, 66, - 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, - 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, - 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, - 97, -1, 99, 100, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, -1, 116, - 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, - 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, - -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, - -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, 203, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, 216, - 217, 218, 219, 220, 221, 222, 223, 224, 225, -1, - -1, 228, 229, 230, 231, -1, 233, 234, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, - 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, 312, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, 352, 353, 354, 355, 356, - 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - 407, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, 452, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, - 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, -1, -1, - -1, -1, -1, -1, 491, 492, 493, -1, -1, -1, - -1, 498, -1, 500, -1, -1, -1, -1, 505, 506, - 507, 508, 3, 4, 5, 6, 7, 8, 9, 10, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, -1, -1, 39, -1, - -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, - 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, - 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, - 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, - 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, - 171, 172, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, 203, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, 216, 217, 218, 219, 220, - 221, 222, 223, 224, 225, -1, -1, 228, 229, 230, - 231, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, - 271, 272, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, 312, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, -1, 406, 407, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, 452, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, - 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, -1, -1, -1, -1, -1, -1, - 491, 492, 493, -1, -1, -1, -1, 498, -1, 500, - -1, -1, -1, -1, 505, 506, 507, 508, 3, 4, + 483, 484, 485, 486, 487, -1, -1, -1, -1, -1, + -1, 494, 495, 496, -1, -1, -1, -1, 501, -1, + 503, -1, -1, -1, -1, 508, 509, 510, 511, 3, + 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, + 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, + 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, + 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, + 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, 232, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, + 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, 487, -1, -1, -1, -1, -1, -1, + 494, 495, 496, -1, -1, -1, -1, 501, -1, 503, + -1, -1, -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, @@ -13438,47 +13272,200 @@ static const yytype_int16 yycheck[] = 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, 203, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, - 225, -1, -1, 228, 229, 230, 231, -1, 233, 234, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, 204, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, - 265, 266, -1, 268, 269, 270, 271, 272, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, 291, 292, 293, 294, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, + 275, 276, 277, 278, 279, -1, 281, 282, -1, -1, + 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, 312, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, 352, 353, 354, + 305, 306, -1, -1, -1, -1, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, + 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, 407, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, 452, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, -1, - -1, -1, -1, -1, -1, -1, 491, 492, -1, -1, - -1, -1, -1, 498, -1, 500, -1, -1, -1, -1, - 505, 506, 507, 508, 3, 4, 5, 6, 7, 8, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, 409, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + 455, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, -1, -1, -1, -1, -1, -1, 494, + 495, 496, -1, -1, -1, -1, 501, -1, 503, -1, + -1, -1, -1, 508, 509, 510, 511, 3, 4, 5, + 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, + 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, + -1, -1, -1, 39, -1, -1, 42, 43, 44, -1, + 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, + 56, 57, -1, 59, 60, 61, 62, 63, 64, 65, + 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, + 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, + 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, + 96, 97, -1, 99, 100, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, -1, + 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, + 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, + 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, + 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, 204, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, + 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, 409, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + 426, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, 455, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, + 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, -1, -1, -1, -1, -1, -1, 494, 495, + 496, -1, -1, -1, -1, 501, -1, 503, -1, -1, + -1, -1, 508, 509, 510, 511, 3, 4, 5, 6, + 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, + 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, + -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, + 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, + 57, -1, 59, 60, 61, 62, 63, 64, 65, 66, + 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, + 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, + 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, + 97, -1, 99, 100, 101, 102, 103, 104, 105, 106, + 107, 108, 109, 110, 111, 112, 113, 114, -1, 116, + 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, + 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, + 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, + 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, + -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, + -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, 189, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, 204, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, + -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, + 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, + 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, + 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, + 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, 314, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, + 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, + 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, 409, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, 426, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, -1, 454, 455, 456, + 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, + 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 487, -1, -1, -1, -1, -1, -1, 494, 495, 496, + -1, -1, -1, -1, 501, -1, 503, -1, -1, -1, + -1, 508, 509, 510, 511, 3, 4, 5, 6, 7, + 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, + 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, + -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, + 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, + -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, + 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, + -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, + 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, + -1, 99, 100, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, -1, 116, 117, + 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, + -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, + 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, + 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, + 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, 204, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, 217, + 218, 219, 220, 221, 222, 223, 224, 225, 226, -1, + -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, + 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, + 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 384, 385, 386, -1, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, 409, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, 426, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, 455, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + -1, -1, -1, -1, -1, -1, 494, 495, 496, -1, + -1, -1, -1, 501, -1, 503, -1, -1, -1, -1, + 508, 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, @@ -13496,141 +13483,91 @@ static const yytype_int16 yycheck[] = 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, 203, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, -1, -1, 218, - 219, 220, 221, 222, 223, 224, 225, -1, -1, 228, - 229, 230, 231, -1, 233, 234, 235, 236, 237, 238, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, 204, 205, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, -1, -1, + 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, 261, 262, 263, 264, 265, 266, -1, 268, - 269, -1, 271, 272, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, + 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, + 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, + 279, -1, 281, 282, -1, -1, 285, 286, 287, -1, + -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, 352, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 382, 383, 384, -1, 386, 387, 388, + 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, -1, 406, 407, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, 452, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, + 399, 400, 401, 402, 403, -1, 405, 406, -1, 408, + 409, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, 423, 424, 425, 426, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, 451, 452, -1, 454, 455, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, -1, -1, -1, -1, - -1, -1, 491, 492, 493, -1, -1, -1, -1, 498, - -1, 500, -1, -1, -1, -1, 505, 506, 507, 508, - 3, 4, 5, 6, 7, -1, 9, 10, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, - 33, -1, -1, -1, -1, -1, 39, -1, -1, 42, - 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, - 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, - 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, - 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - 203, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, 216, 217, 218, 219, 220, 221, 222, - 223, 224, 225, -1, -1, 228, 229, 230, 231, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, - 263, 264, 265, 266, -1, 268, 269, 270, 271, 272, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, 312, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, 352, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, 407, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, 452, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, -1, -1, -1, -1, -1, -1, -1, 491, 492, - -1, -1, -1, -1, -1, 498, -1, 500, -1, -1, - -1, -1, 505, 506, 507, 508, 3, 4, 5, 6, - 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, - 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, - -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, - 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, - 57, -1, 59, 60, 61, 62, 63, 64, 65, 66, - 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, - 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, - 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, - 97, -1, 99, 100, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, - 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, - 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, - -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, - -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, 203, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, -1, - 217, 218, 219, 220, 221, 222, 223, 224, 225, -1, - -1, 228, 229, 230, 231, -1, 233, 234, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, - -1, 268, 269, 270, 271, 272, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, 291, 292, 293, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, 312, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, 352, 353, 354, 355, 356, - 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - 407, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, 423, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, 452, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, - 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, -1, -1, -1, - -1, -1, -1, -1, 491, 492, -1, -1, -1, -1, - -1, 498, -1, 500, -1, -1, -1, -1, 505, 506, - 507, 508, 3, 4, 5, 6, 7, -1, 9, 10, + 479, 480, 481, 482, 483, 484, 485, 486, 487, -1, + -1, -1, -1, -1, -1, 494, 495, 496, -1, -1, + -1, -1, 501, -1, 503, -1, -1, -1, -1, 508, + 509, 510, 511, 3, 4, 5, 6, 7, 8, 9, + 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, -1, -1, -1, -1, -1, 39, + -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, + 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, + 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, + -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, + 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, -1, -1, 229, + 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, 262, 263, 264, 265, 266, 267, -1, 269, + 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, -1, 405, 406, -1, 408, 409, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, 426, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, 455, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, -1, -1, -1, + -1, -1, -1, -1, 494, 495, -1, -1, -1, -1, + -1, 501, -1, 503, -1, -1, -1, -1, 508, 509, + 510, 511, 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, 39, -1, @@ -13641,100 +13578,53 @@ static const yytype_int16 yycheck[] = 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, + 111, 112, 113, 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, - 171, 172, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, 203, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, 216, 217, 218, 219, 220, - 221, 222, 223, 224, 225, -1, -1, 228, 229, 230, - 231, -1, 233, 234, 235, 236, 237, 238, 239, 240, + 171, 172, 173, -1, 175, -1, 177, 178, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, 204, 205, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, -1, -1, 219, 220, + 221, 222, 223, 224, 225, 226, -1, -1, 229, 230, + 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - 261, 262, 263, 264, 265, 266, -1, 268, 269, 270, - 271, 272, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, + 261, 262, 263, 264, 265, 266, 267, -1, 269, 270, + -1, 272, 273, 274, 275, 276, 277, 278, 279, -1, + 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, 312, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, -1, 406, 407, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, + 401, 402, 403, -1, 405, 406, -1, 408, 409, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, 452, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, + 431, 432, 433, 434, -1, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + 451, 452, -1, 454, 455, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, -1, -1, -1, -1, -1, -1, 3, - 4, 5, 6, 7, 8, 9, 10, 498, -1, 500, - -1, -1, -1, -1, 505, 506, 507, 508, 22, 23, - 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, - -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, - 44, -1, 46, 47, 48, 49, 50, 51, 52, -1, - 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, - 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, - 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, - 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, - 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, - 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, - 134, -1, 136, 137, 138, -1, 140, -1, 142, -1, - 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, - 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, - -1, 165, 166, -1, 168, 169, 170, 171, -1, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, -1, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, 203, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, -1, -1, 218, 219, 220, 221, 222, 223, - 224, 225, -1, -1, 228, 229, 230, -1, -1, 233, - 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, - 264, 265, 266, -1, 268, 269, -1, 271, 272, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, -1, - 284, 285, 286, -1, -1, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, 312, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, 352, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, -1, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, -1, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, 407, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, 452, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - -1, -1, -1, -1, -1, -1, -1, 491, 492, 493, - -1, 3, 4, 5, 498, -1, 500, 9, -1, -1, - -1, 505, 506, 507, 508, -1, -1, -1, -1, -1, + 481, 482, 483, 484, 485, 486, 487, -1, -1, -1, + -1, -1, -1, 494, 495, 496, -1, -1, -1, -1, + 501, -1, 503, -1, -1, -1, -1, 508, 509, 510, + 511, 3, 4, 5, 6, 7, -1, 9, 10, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, - 32, 33, -1, -1, -1, 37, -1, -1, -1, -1, + 32, 33, -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, - 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, + 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, @@ -13745,287 +13635,295 @@ static const yytype_int16 yycheck[] = 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, - 172, 173, -1, 175, -1, 177, 178, 179, -1, 181, + 172, 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, -1, -1, 228, 229, 230, 231, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, 204, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, -1, -1, 229, 230, 231, + 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 264, 265, 266, -1, 268, 269, 270, 271, - -1, 273, 274, 275, 276, 277, 278, -1, 280, 281, - 282, -1, 284, 285, 286, -1, -1, 289, 290, 291, + 262, 263, 264, 265, 266, 267, -1, 269, 270, 271, + 272, 273, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - 312, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, 314, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, 369, 370, 371, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, + 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, 407, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, + 402, 403, -1, 405, 406, -1, 408, 409, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - 452, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, 455, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, -1, -1, -1, -1, -1, -1, -1, 491, - 492, 493, -1, 3, 4, 5, 6, 7, 500, 9, - -1, -1, -1, -1, -1, -1, 508, -1, -1, -1, - -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, -1, -1, 39, - -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, - 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, - 60, 61, 62, 63, 64, 65, -1, 67, 68, 69, - 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, - -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, - 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, - 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, - 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, - 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, - 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, - 170, 171, 172, 173, -1, 175, 176, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, 188, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, 203, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, 216, 217, 218, 219, - 220, 221, 222, 223, 224, 225, -1, -1, 228, 229, - 230, 231, -1, 233, 234, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, 261, 262, 263, 264, 265, 266, -1, 268, 269, - 270, 271, 272, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, 312, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 382, 383, 384, -1, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, 407, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, 452, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, - 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, -1, -1, -1, -1, 3, 4, - 5, 6, 7, -1, 9, -1, -1, -1, 498, -1, - 500, -1, -1, -1, -1, 505, 506, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, -1, -1, 39, -1, -1, 42, 43, 44, - -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, - 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, - 65, -1, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, - 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, 203, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, - 225, -1, -1, 228, 229, 230, 231, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, - 265, 266, -1, 268, 269, 270, 271, 272, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, 312, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, 352, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, 407, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, 452, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, -1, - -1, -1, -1, 3, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 498, -1, 500, -1, -1, -1, -1, - 505, 506, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, - 40, -1, -1, 43, 44, -1, 46, 47, 48, -1, - 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, - 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, - 70, 71, -1, 73, 74, 75, 76, 77, -1, -1, - -1, 81, 82, 83, 84, 85, 86, -1, 88, 89, - 90, -1, 92, 93, 94, 95, 96, 97, -1, -1, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, - 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, - 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, - 140, 141, 142, 143, 144, 145, 146, 147, 148, 149, - 150, 151, 152, 153, 154, -1, 156, 157, 158, 159, - 160, 161, 162, 163, -1, 165, -1, -1, -1, 169, - 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, -1, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, -1, 204, 205, 206, 207, 208, 209, - 210, 211, 212, 213, -1, 215, -1, -1, 218, -1, - 220, 221, 222, 223, 224, 225, -1, -1, 228, -1, - 230, -1, 232, 233, 234, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, -1, 262, 263, 264, 265, 266, -1, 268, 269, - -1, 271, -1, 273, 274, 275, 276, 277, 278, 279, - 280, 281, -1, -1, 284, 285, 286, -1, 288, 289, - 290, -1, 292, -1, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, -1, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, -1, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - -1, 381, 382, 383, 384, -1, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, -1, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, -1, 424, 425, 426, 427, 428, 429, - 430, 431, 432, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, -1, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, - 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 3, -1, 5, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 501, -1, 22, 23, 24, 25, 26, 27, 28, - 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, - -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, + 482, 483, 484, 485, 486, -1, -1, -1, -1, -1, + -1, -1, 494, 495, -1, -1, -1, -1, -1, 501, + -1, 503, -1, -1, -1, -1, 508, 509, 510, 511, + 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, + 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, + 33, -1, -1, -1, -1, -1, 39, -1, -1, 42, + 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, + 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, + 63, 64, 65, 66, 67, 68, 69, 70, 71, -1, + 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, + 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, + 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, + 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, + 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, + 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, + 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, + 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, + 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, 204, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, -1, 218, 219, 220, 221, 222, + 223, 224, 225, 226, -1, -1, 229, 230, 231, 232, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, + 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, + 263, 264, 265, 266, 267, -1, 269, 270, 271, 272, + 273, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, + 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, -1, -1, -1, -1, 311, 312, + 313, 314, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, + 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, -1, 408, 409, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, 426, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, -1, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, 455, 456, 457, 458, 459, 460, -1, -1, + 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, + 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, -1, -1, -1, -1, -1, -1, + -1, 494, 495, -1, -1, -1, -1, -1, 501, -1, + 503, -1, -1, -1, -1, 508, 509, 510, 511, 3, + 4, 5, 6, 7, -1, 9, 10, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, + 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, + 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, + 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, + 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, 232, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, + 264, 265, 266, 267, -1, 269, 270, 271, 272, 273, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, -1, -1, -1, -1, -1, -1, 3, + 4, 5, 6, 7, 8, 9, 10, 501, -1, 503, + -1, -1, -1, -1, 508, 509, 510, 511, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, + 44, -1, 46, 47, 48, 49, 50, 51, 52, -1, + 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, + 64, 65, 66, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, + 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, -1, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, 166, -1, 168, 169, 170, 171, -1, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, -1, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, -1, -1, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, -1, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, + 264, 265, 266, 267, -1, 269, 270, -1, 272, 273, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, -1, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, -1, -1, -1, -1, -1, -1, -1, + 494, 495, 496, -1, 3, 4, 5, 501, -1, 503, + 9, -1, -1, -1, 508, 509, 510, 511, -1, -1, + -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, + 29, -1, 31, 32, 33, -1, -1, -1, 37, -1, + -1, -1, -1, 42, 43, 44, -1, 46, 47, 48, + 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, - -1, -1, 81, 82, 83, 84, 85, 86, -1, 88, + 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, - -1, 100, 101, 102, 103, 104, 105, 106, 107, 108, + 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, - 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, - 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, - 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, -1, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, -1, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, -1, -1, 218, - -1, 220, 221, 222, 223, 224, 225, -1, -1, 228, - -1, 230, -1, -1, 233, 234, 235, 236, 237, 238, + 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, + 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, + 169, 170, 171, 172, 173, -1, 175, -1, 177, 178, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, 204, 205, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, -1, -1, + 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, -1, 262, 263, 264, 265, 266, -1, 268, - 269, -1, 271, -1, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, -1, 292, -1, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, -1, 313, 314, 315, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, + 259, 260, 261, 262, 263, 264, 265, 266, 267, -1, + 269, 270, 271, 272, -1, 274, 275, 276, 277, 278, + 279, -1, 281, 282, 283, -1, 285, 286, 287, -1, + -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, -1, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, -1, 381, 382, 383, 384, -1, 386, 387, 388, + 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, -1, 406, -1, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, -1, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, -1, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, + 399, 400, 401, 402, 403, -1, 405, 406, -1, 408, + 409, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, 423, 424, 425, 426, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, 451, 452, -1, 454, 455, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 3, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 500, 501, -1, 22, 23, 24, 25, 26, 27, - 28, 29, -1, 31, 32, 33, 34, 35, -1, 37, - -1, -1, -1, -1, 42, 43, 44, -1, 46, 47, - 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, - -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, - 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, - -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, - 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, - -1, 99, 100, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, - 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, - -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, - 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, - 158, 159, 160, 161, -1, 163, -1, 165, 166, 167, - 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, 203, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, 216, 217, - 218, 219, 220, 221, 222, 223, 224, 225, -1, -1, - 228, 229, 230, 231, -1, 233, 234, 235, 236, 237, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, 261, 262, 263, 264, 265, 266, -1, - 268, 269, 270, 271, -1, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, 291, 292, 293, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - -1, 309, 310, 311, 312, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, 352, 353, 354, 355, 356, 357, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 382, 383, 384, -1, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, 405, 406, 407, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, -1, 424, 425, 426, 427, - 428, 429, 430, 431, -1, -1, 434, 435, 436, 437, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, 452, 453, 454, 455, 456, 457, - -1, -1, 460, -1, 462, 463, 464, 465, 466, 467, - 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, -1, 3, 486, 5, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 500, -1, -1, -1, 22, 23, 24, 25, + 479, 480, 481, 482, 483, 484, 485, 486, -1, -1, + -1, -1, -1, -1, -1, 494, 495, 496, -1, 3, + 4, 5, 6, 7, 503, 9, -1, -1, -1, -1, + -1, -1, 511, -1, -1, -1, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, -1, -1, 39, -1, -1, 42, 43, + 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, + 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, + 64, 65, -1, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, + 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, + -1, 175, 176, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, 232, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, + 264, 265, 266, 267, -1, 269, 270, 271, 272, 273, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, -1, -1, -1, -1, 3, 4, 5, + 6, 7, -1, 9, -1, -1, -1, 501, -1, 503, + -1, -1, -1, -1, 508, 509, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, - -1, -1, -1, -1, -1, -1, 42, 43, 44, -1, + -1, -1, -1, 39, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, - 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, - 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, + 56, 57, -1, 59, 60, 61, 62, 63, 64, 65, + -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, 105, @@ -14033,138 +13931,333 @@ static const yytype_int16 yycheck[] = 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, + 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, - 166, 167, 168, 169, 170, 171, 172, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, 203, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, + 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, 204, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, - 226, -1, 228, 229, 230, 231, -1, 233, 234, 235, + 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, - 266, -1, 268, 269, 270, 271, -1, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, 283, 284, 285, - 286, -1, -1, 289, 290, 291, 292, 293, 294, 295, + 266, 267, -1, 269, 270, 271, 272, 273, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, -1, 309, 310, 311, 312, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, 352, 353, 354, 355, + 306, -1, -1, -1, -1, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, 368, -1, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, 405, - 406, 407, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, -1, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, 437, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, 452, 453, 454, 455, - 456, 457, -1, -1, 460, -1, 462, 463, 464, 465, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, 409, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + 426, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, 455, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, -1, 3, - 486, 5, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 500, -1, -1, -1, 22, 23, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, -1, -1, -1, -1, 3, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 501, -1, 503, -1, -1, + -1, -1, 508, 509, 22, 23, 24, 25, 26, 27, + 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, + -1, -1, 40, -1, -1, 43, 44, -1, 46, 47, + 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, + -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, + 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, + -1, -1, -1, 81, 82, 83, 84, 85, 86, -1, + 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, + -1, -1, 100, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, + 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, + -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, + 138, -1, 140, 141, 142, 143, 144, 145, 146, 147, + 148, 149, 150, 151, 152, 153, 154, -1, 156, 157, + 158, 159, 160, 161, 162, 163, -1, 165, -1, -1, + -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, -1, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, -1, 205, 206, 207, + 208, 209, 210, 211, 212, 213, 214, -1, 216, -1, + -1, 219, -1, 221, 222, 223, 224, 225, 226, -1, + -1, 229, -1, 231, -1, 233, 234, 235, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 258, 259, 260, 261, -1, 263, 264, 265, 266, 267, + -1, 269, 270, -1, 272, -1, 274, 275, 276, 277, + 278, 279, 280, 281, 282, -1, -1, 285, 286, 287, + -1, 289, 290, 291, 292, -1, 294, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, -1, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, + 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, -1, 383, 384, 385, 386, -1, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, -1, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, -1, 427, + 428, 429, 430, 431, 432, 433, 434, 435, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, -1, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, + -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 504, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, - -1, -1, -1, -1, -1, -1, -1, -1, 42, 43, - 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, - 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, - 64, -1, 66, 67, 68, 69, 70, 71, -1, 73, - 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, - 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, - 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, + 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, + 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, + 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, -1, -1, 81, 82, 83, + 84, 85, 86, -1, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, - -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, 188, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, 203, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, 216, 217, 218, 219, 220, 221, 222, 223, - 224, 225, 226, -1, 228, 229, 230, 231, -1, 233, + -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, -1, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + -1, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, -1, -1, 219, -1, 221, 222, 223, + 224, 225, 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, - 264, 265, 266, -1, 268, 269, 270, 271, -1, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, 283, - 284, 285, 286, -1, -1, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, 312, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, 352, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, -1, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, 407, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, -1, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, 452, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, + 254, 255, 256, 257, 258, 259, 260, 261, -1, 263, + 264, 265, 266, 267, -1, 269, 270, -1, 272, -1, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, -1, + 294, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + -1, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, -1, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, -1, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, -1, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, -1, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 500, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, - 33, -1, -1, -1, -1, -1, -1, -1, -1, 42, - 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, - 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, - 63, 64, -1, 66, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, - 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - 203, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, 216, 217, 218, 219, 220, 221, 222, - 223, 224, 225, -1, -1, 228, 229, 230, 231, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, - 263, 264, 265, 266, -1, 268, 269, 270, 271, -1, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, 312, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, 352, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, 407, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - -1, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, 452, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 3, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 500, -1, -1, + 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 503, + 504, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, 34, 35, -1, 37, -1, -1, + -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, + 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, + 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, + -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, 166, 167, 168, 169, + 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, -1, -1, 229, + 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, 262, 263, 264, 265, 266, 267, -1, 269, + 270, 271, 272, -1, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, + -1, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, -1, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, -1, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + 440, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, 455, 456, 457, 458, 459, + 460, -1, -1, 463, -1, 465, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, -1, 3, 489, + 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 503, -1, -1, -1, 22, 23, 24, + 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, + -1, -1, -1, -1, -1, -1, -1, 42, 43, 44, + -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, + 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, + -1, 66, 67, 68, 69, 70, 71, -1, 73, 74, + 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, + 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, + 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, + -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, + 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, + -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, + -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, + 165, 166, 167, 168, 169, 170, 171, 172, 173, -1, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, 204, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 226, 227, -1, 229, 230, 231, 232, -1, 234, + 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, + 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, + 265, 266, 267, -1, 269, 270, 271, 272, -1, 274, + 275, 276, 277, 278, 279, -1, 281, 282, -1, 284, + 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, + 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, -1, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, + 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, -1, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, -1, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, 440, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + 455, 456, 457, 458, 459, 460, -1, -1, 463, -1, + 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, -1, 3, 489, 5, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 503, -1, + -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, + -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, + 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, + 60, 61, 62, 63, 64, -1, 66, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, + -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, + 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, 227, -1, 229, + 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, 262, 263, 264, 265, 266, 267, -1, 269, + 270, 271, 272, -1, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, 284, 285, 286, 287, -1, -1, + 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, -1, 405, 406, -1, 408, 409, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, -1, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, 455, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 3, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 503, -1, -1, 22, 23, 24, 25, + 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, + -1, -1, -1, -1, -1, -1, 42, 43, 44, -1, + 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, + 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, + 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, + 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, + 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, + 96, 97, -1, 99, 100, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, + 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, + 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, + 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, + 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, 204, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, + 266, 267, -1, 269, 270, 271, 272, -1, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, + 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, 409, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + -1, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, 455, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, + 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 503, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, @@ -14180,137 +14273,41 @@ static const yytype_int16 yycheck[] = 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, - 172, 173, -1, 175, -1, 177, 178, 179, -1, 181, + 172, 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, -1, -1, 228, 229, 230, 231, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, 204, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, -1, -1, 229, 230, 231, + 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 264, 265, 266, -1, 268, 269, 270, 271, - -1, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, 291, + 262, 263, 264, 265, 266, 267, -1, 269, 270, 271, + 272, -1, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - 312, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, 314, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, -1, 370, 371, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, + 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, 407, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, -1, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - 452, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, + 402, 403, -1, 405, 406, -1, 408, 409, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, -1, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, 455, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 3, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 500, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, 30, - 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, - 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, - 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, 77, -1, -1, -1, - 81, 82, 83, 84, 85, 86, -1, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, -1, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, - 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, -1, 167, -1, 169, 170, - 171, -1, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, -1, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, -1, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, -1, -1, 218, -1, 220, - 221, 222, 223, 224, 225, -1, -1, 228, -1, 230, - -1, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - -1, 262, 263, 264, 265, 266, -1, 268, 269, -1, - 271, -1, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - -1, 292, -1, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, -1, 309, 310, - 311, -1, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, -1, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, 368, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, -1, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, 405, 406, -1, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, -1, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, 437, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, -1, 453, 454, 455, 456, 457, -1, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, 469, 470, - 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 3, -1, 486, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 500, - -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, - 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, - 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, - 70, 71, -1, 73, 74, 75, 76, 77, -1, -1, - -1, 81, 82, 83, 84, 85, 86, -1, 88, 89, - 90, -1, 92, 93, 94, 95, 96, 97, -1, -1, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, - 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, - 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, - 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, - 160, 161, -1, 163, -1, 165, -1, 167, -1, 169, - 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, -1, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, -1, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, -1, -1, 218, -1, - 220, 221, 222, 223, 224, 225, -1, -1, 228, -1, - 230, -1, -1, 233, 234, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, -1, 262, 263, 264, 265, 266, -1, 268, 269, - -1, 271, -1, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, -1, 292, -1, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, -1, 309, - 310, 311, -1, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, -1, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, 368, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - -1, 381, 382, 383, 384, -1, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, 405, 406, -1, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, -1, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, 437, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, -1, 453, 454, 455, 456, 457, -1, -1, - 460, -1, 462, 463, 464, 465, 466, 467, 468, 469, - 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, -1, 3, 486, 5, -1, -1, + 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 500, -1, -1, -1, 22, 23, 24, 25, 26, 27, - 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, + -1, 503, -1, -1, 22, 23, 24, 25, 26, 27, + 28, 29, 30, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, @@ -14323,41 +14320,235 @@ static const yytype_int16 yycheck[] = -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, - 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, + 158, 159, 160, 161, -1, 163, -1, 165, -1, 167, -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - -1, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, -1, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, -1, -1, - 218, -1, 220, 221, 222, 223, 224, 225, -1, -1, - 228, -1, 230, -1, -1, 233, 234, 235, 236, 237, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, -1, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, -1, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, -1, + -1, 219, -1, 221, 222, 223, 224, 225, 226, -1, + -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, -1, 262, 263, 264, 265, 266, -1, - 268, 269, -1, 271, -1, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, -1, 292, -1, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, -1, -1, -1, - -1, 309, 310, 311, -1, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 258, 259, 260, 261, -1, 263, 264, 265, 266, 267, + -1, 269, 270, -1, 272, -1, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, -1, 294, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, -1, 311, 312, 313, -1, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, -1, 353, 354, 355, 356, 357, + 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - -1, -1, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, -1, 381, 382, 383, 384, -1, 386, 387, + 368, 369, 370, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, -1, 406, -1, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, -1, 424, 425, 426, 427, - 428, 429, 430, 431, -1, -1, 434, 435, 436, -1, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, -1, 453, 454, 455, 456, 457, - -1, -1, 460, -1, -1, 463, 464, 465, 466, 467, + 398, 399, 400, 401, 402, 403, -1, 405, 406, 407, + 408, -1, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, -1, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, 440, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, -1, 456, 457, + 458, 459, 460, -1, -1, 463, -1, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 3, -1, 5, -1, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, + -1, 489, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 503, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, + 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, + 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, + 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, -1, -1, 81, 82, 83, + 84, 85, 86, -1, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, -1, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, -1, 167, -1, 169, 170, 171, -1, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, -1, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + -1, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, -1, -1, 219, -1, 221, 222, 223, + 224, 225, 226, -1, -1, 229, -1, 231, -1, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, -1, 263, + 264, 265, 266, 267, -1, 269, 270, -1, 272, -1, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, -1, + 294, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 309, -1, 311, 312, 313, + -1, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, 370, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, -1, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, 407, 408, -1, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, -1, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, 440, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, -1, 456, 457, 458, 459, 460, -1, -1, 463, + -1, 465, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, -1, 3, 489, 5, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 503, + -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, + 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, + -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, + 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, + 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, + -1, -1, 81, 82, 83, 84, 85, 86, -1, 88, + 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, + -1, 100, 101, 102, 103, 104, 105, 106, 107, 108, + 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, + 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, + -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, + -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, + 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, + 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, + 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + -1, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, -1, 205, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, -1, -1, + 219, -1, 221, 222, 223, 224, 225, 226, -1, -1, + 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, + 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, + 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, + 259, 260, 261, -1, 263, 264, 265, 266, 267, -1, + 269, 270, -1, 272, -1, 274, 275, 276, 277, 278, + 279, -1, 281, 282, -1, -1, 285, 286, 287, -1, + -1, 290, 291, 292, -1, 294, -1, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, 313, -1, 315, 316, 317, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, + 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, -1, 383, 384, 385, 386, -1, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 402, 403, -1, 405, 406, -1, 408, + -1, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, 423, 424, 425, -1, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, 451, 452, -1, 454, -1, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, + 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, + 479, 480, 481, 482, 483, 484, 485, 486, 3, -1, + 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 503, -1, -1, 22, 23, 24, + 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, + -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, + -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, + -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, + 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, + 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, + 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, + -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, + 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, + -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, + -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, + 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, -1, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, -1, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, -1, -1, 219, -1, 221, 222, 223, 224, + 225, 226, -1, -1, 229, -1, 231, -1, -1, 234, + 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, + 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 258, 259, 260, 261, -1, 263, 264, + 265, 266, 267, -1, 269, 270, -1, 272, -1, 274, + 275, 276, 277, 278, 279, -1, 281, 282, -1, -1, + 285, 286, 287, -1, -1, 290, 291, 292, -1, 294, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, -1, -1, -1, -1, 311, 312, 313, -1, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, -1, + 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, -1, 383, 384, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, -1, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, -1, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + -1, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 503, -1, + -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, + 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, + 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, + 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, + 71, -1, 73, 74, 75, 76, 77, -1, -1, -1, + 81, 82, 83, 84, 85, 86, -1, 88, 89, 90, + -1, 92, 93, 94, 95, 96, 97, -1, -1, 100, + 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, + 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, + 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, + 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, + 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, + 171, -1, 173, -1, 175, -1, 177, 178, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, -1, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, -1, 205, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, -1, -1, 219, -1, + 221, 222, 223, 224, 225, 226, -1, -1, 229, -1, + 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, + 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, + 261, -1, 263, 264, 265, 266, 267, -1, 269, 270, + -1, 272, -1, 274, 275, 276, 277, 278, 279, -1, + 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, + 291, 292, -1, 294, -1, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, 313, -1, 315, 316, 317, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 402, 403, -1, 405, 406, -1, 408, -1, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, + 421, 422, 423, 424, 425, -1, 427, 428, 429, 430, + 431, 432, 433, 434, -1, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + 451, 452, -1, 454, -1, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, + 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 500, -1, -1, 22, 23, 24, 25, 26, + -1, -1, 503, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, @@ -14373,183 +14564,39 @@ static const yytype_int16 yycheck[] = 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, -1, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, -1, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, -1, - -1, 218, -1, 220, 221, 222, 223, 224, 225, -1, - -1, 228, -1, 230, -1, -1, 233, 234, 235, 236, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, -1, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, -1, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + -1, -1, 219, -1, 221, 222, 223, 224, 225, 226, + -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, -1, 262, 263, 264, 265, 266, - -1, 268, 269, -1, 271, -1, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, -1, 292, -1, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, -1, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, -1, 353, 354, 355, 356, + 257, 258, 259, 260, 261, -1, 263, 264, 265, 266, + 267, -1, 269, 270, -1, 272, -1, 274, 275, 276, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, -1, 294, -1, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, -1, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, -1, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - -1, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, -1, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, -1, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, + 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, -1, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, -1, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, -1, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, -1, 454, -1, 456, + 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 3, -1, 5, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 500, -1, -1, 22, 23, 24, 25, - 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, - 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, - 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, - -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, - 76, 77, -1, -1, -1, 81, 82, 83, 84, 85, - 86, -1, 88, 89, 90, -1, 92, 93, 94, 95, - 96, 97, -1, -1, 100, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, - 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, - 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, - 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, - -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, -1, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, -1, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, - -1, -1, 218, -1, 220, 221, 222, 223, 224, 225, - -1, -1, 228, -1, 230, -1, -1, 233, 234, 235, - 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, -1, 262, 263, 264, 265, - 266, -1, 268, 269, -1, 271, -1, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, -1, 284, 285, - 286, -1, -1, 289, 290, -1, 292, -1, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, -1, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, -1, 353, 354, 355, - 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, -1, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, -1, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, -1, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, -1, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, -1, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, - 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 3, -1, - 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 500, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, - -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, - -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, - -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, - 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, -1, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, -1, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, -1, -1, 218, -1, 220, 221, 222, 223, 224, - 225, -1, -1, 228, -1, 230, -1, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, -1, 262, 263, 264, - 265, 266, -1, 268, 269, -1, 271, -1, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, -1, 292, -1, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, -1, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, -1, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, -1, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, -1, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, -1, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, -1, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 3, - -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 500, -1, -1, 22, 23, - 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, - 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, - 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, - 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, - 74, 75, 76, 77, -1, -1, -1, 81, 82, 83, - 84, 85, 86, -1, 88, 89, 90, -1, 92, 93, - 94, 95, 96, 97, -1, -1, 100, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, - 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, - 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, - 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, - 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, - -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, -1, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, -1, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, -1, -1, 218, -1, 220, 221, 222, 223, - 224, 225, -1, -1, 228, -1, 230, -1, -1, 233, - 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, -1, 262, 263, - 264, 265, 266, -1, 268, 269, -1, 271, -1, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, -1, - 284, 285, 286, -1, -1, 289, 290, -1, 292, -1, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, -1, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, -1, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, -1, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, -1, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, -1, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, -1, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, -1, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 500, -1, -1, 22, + -1, -1, -1, -1, -1, -1, 503, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, @@ -14565,87 +14612,136 @@ static const yytype_int16 yycheck[] = -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, -1, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - -1, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, -1, -1, 218, -1, 220, 221, 222, - 223, 224, 225, -1, -1, 228, -1, 230, -1, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, + 183, 184, 185, 186, 187, 188, -1, 190, 191, 192, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, -1, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, -1, -1, 219, -1, 221, 222, + 223, 224, 225, 226, -1, -1, 229, -1, 231, -1, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, -1, 262, - 263, 264, 265, 266, -1, 268, 269, -1, 271, -1, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, -1, 292, - -1, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, -1, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, -1, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, -1, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, -1, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - -1, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, -1, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, + 253, 254, 255, 256, 257, 258, 259, 260, 261, -1, + 263, 264, 265, 266, 267, -1, 269, 270, -1, 272, + -1, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, + -1, 294, -1, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, -1, -1, -1, -1, 311, 312, + 313, -1, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, + 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, -1, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, -1, 408, -1, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, -1, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, -1, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, -1, 456, 457, 458, 459, 460, -1, -1, + 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 3, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 500, -1, -1, - 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, - 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, - 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, - 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, - -1, 73, 74, 75, 76, 77, -1, -1, -1, 81, - 82, 83, 84, 85, 86, -1, 88, 89, 90, -1, - 92, 93, 94, 95, 96, 97, -1, -1, 100, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, - -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, - 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, - 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, - -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, - -1, 173, -1, 175, -1, 177, 178, 179, -1, 181, - 182, 183, 184, 185, 186, 187, -1, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, -1, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, -1, -1, 218, -1, 220, 221, - 222, 223, 224, 225, -1, -1, 228, -1, 230, -1, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, -1, - 262, 263, 264, 265, 266, -1, 268, 269, -1, 271, - -1, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, -1, - 292, -1, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - -1, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - -1, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, -1, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, -1, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, -1, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, -1, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - -1, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, - 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 3, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 500, -1, + 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 503, -1, -1, 22, 23, 24, 25, 26, 27, 28, + 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, + -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, + 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, + 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, + -1, -1, 81, 82, 83, 84, 85, 86, -1, 88, + 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, + -1, 100, 101, 102, 103, 104, 105, 106, 107, 108, + 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, + 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, + -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, + -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, + 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, + 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, + 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + -1, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, -1, 205, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, -1, -1, + 219, -1, 221, 222, 223, 224, 225, 226, -1, -1, + 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, + 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, + 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, + 259, 260, 261, -1, 263, 264, 265, 266, 267, -1, + 269, 270, -1, 272, -1, 274, 275, 276, 277, 278, + 279, -1, 281, 282, -1, -1, 285, 286, 287, -1, + -1, 290, 291, 292, -1, 294, -1, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, 313, -1, 315, 316, 317, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, + 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, -1, 383, 384, 385, 386, -1, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 402, 403, -1, 405, 406, -1, 408, + -1, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, 423, 424, 425, -1, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, 451, 452, -1, 454, -1, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, + 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, + 479, 480, 481, 482, 483, 484, 485, 486, 3, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 503, -1, -1, 22, 23, 24, + 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, + -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, + -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, + -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, + 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, + 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, + 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, + -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, + 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, + -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, + -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, + 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, -1, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, -1, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, -1, -1, 219, -1, 221, 222, 223, 224, + 225, 226, -1, -1, 229, -1, 231, -1, -1, 234, + 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, + 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 258, 259, 260, 261, -1, 263, 264, + 265, 266, 267, -1, 269, 270, -1, 272, -1, 274, + 275, 276, 277, 278, 279, -1, 281, 282, -1, -1, + 285, 286, 287, -1, -1, 290, 291, 292, -1, 294, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, -1, -1, -1, -1, 311, 312, 313, -1, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, -1, + 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, -1, 383, 384, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, -1, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, -1, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + -1, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 503, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, @@ -14661,465 +14757,437 @@ static const yytype_int16 yycheck[] = 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, - 171, -1, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, -1, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, -1, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, -1, -1, 218, -1, 220, - 221, 222, 223, 224, 225, -1, -1, 228, -1, 230, - -1, -1, 233, 234, 235, 236, 237, 238, 239, 240, + 171, -1, 173, -1, 175, -1, 177, 178, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, -1, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, -1, 205, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, -1, -1, 219, -1, + 221, 222, 223, 224, 225, 226, -1, -1, 229, -1, + 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - -1, 262, 263, 264, 265, 266, -1, 268, 269, -1, - 271, -1, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - -1, 292, -1, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, -1, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, -1, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, -1, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, + 261, -1, 263, 264, 265, 266, 267, -1, 269, 270, + -1, 272, -1, 274, 275, 276, 277, 278, 279, -1, + 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, + 291, 292, -1, 294, -1, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, 313, -1, 315, 316, 317, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, -1, 406, -1, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, -1, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, -1, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, + 401, 402, 403, -1, 405, 406, -1, 408, -1, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, + 421, 422, 423, 424, 425, -1, 427, 428, 429, 430, + 431, 432, 433, 434, -1, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + 451, 452, -1, 454, -1, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 3, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 500, - -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, - 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, - 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, - 70, 71, -1, 73, 74, 75, 76, 77, -1, -1, - -1, 81, 82, 83, 84, 85, 86, -1, 88, 89, - 90, -1, 92, 93, 94, 95, 96, 97, -1, -1, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, - 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, - 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, - 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, - 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, - 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, -1, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, -1, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, -1, -1, 218, -1, - 220, 221, 222, 223, 224, 225, -1, -1, 228, -1, - 230, -1, -1, 233, 234, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, -1, 262, 263, 264, 265, 266, -1, 268, 269, - -1, 271, -1, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, -1, 292, -1, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, -1, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, -1, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - -1, 381, 382, 383, 384, -1, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, -1, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, -1, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, -1, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, - 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, -1, 3, 4, 5, -1, -1, - 8, 9, -1, -1, -1, -1, -1, 15, 16, -1, - 500, 19, 20, 21, 22, 23, 24, 25, 26, 27, - 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, - 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, - 48, 49, 50, 51, 52, -1, 54, 55, 56, 57, - 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, - 68, 69, 70, 71, 72, 73, 74, 75, 76, -1, - 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, - 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, - 98, 99, 100, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, 113, 114, 115, 116, 117, - 118, 119, 120, 121, 122, 123, 124, 125, 126, 127, - 128, 129, 130, 131, 132, 133, 134, 135, 136, 137, - 138, 139, 140, 141, 142, 143, 144, 145, 146, 147, - -1, 149, 150, 151, 152, 153, -1, 155, 156, 157, - 158, 159, 160, 161, 162, 163, 164, 165, 166, 167, - 168, 169, 170, 171, 172, 173, 174, 175, 176, -1, - -1, 179, 180, 181, 182, 183, 184, 185, 186, 187, - 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, - 198, 199, 200, 201, 202, 203, -1, 205, 206, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 222, 223, 224, 225, 226, 227, - 228, 229, 230, 231, 232, 233, 234, 235, 236, 237, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, -1, 274, 275, 276, 277, - 278, 279, 280, 281, 282, 283, 284, 285, 286, 287, - 288, 289, -1, 291, 292, 293, -1, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, -1, 312, 313, 314, -1, 316, 317, - 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, - 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, -1, 401, 402, 403, 404, 405, 406, 407, - 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 419, -1, 421, -1, 423, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, - -1, 449, 450, 451, 452, 453, 454, 455, 456, 457, - 458, 459, 460, 461, 462, 463, 464, 465, 466, 467, - 468, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 480, 481, 482, 483, -1, 3, -1, 487, - 488, 489, 8, 491, 492, 493, 494, 495, 496, 15, - 16, -1, -1, 19, 20, 21, 22, 23, 24, 25, - 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, - 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, - 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, - -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, - 76, 77, -1, -1, -1, 81, 82, 83, 84, 85, - 86, -1, 88, 89, 90, -1, 92, 93, 94, 95, - 96, 97, -1, -1, 100, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, - 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, - 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, - 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, - -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, -1, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, -1, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, - -1, -1, 218, -1, 220, 221, 222, 223, 224, 225, - -1, -1, 228, -1, 230, -1, -1, 233, 234, 235, - 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, -1, 262, 263, 264, 265, - 266, -1, 268, 269, -1, 271, -1, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, -1, 284, 285, - 286, -1, -1, 289, 290, -1, 292, -1, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, -1, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, -1, 353, 354, 355, - 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, -1, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, -1, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, -1, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, -1, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, -1, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, - 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, -1, -1, - -1, 487, 488, 489, -1, 491, 492, 493, 494, 495, - 496, 8, -1, -1, 11, -1, -1, -1, 15, 16, - 17, 18, 19, 20, 21, -1, -1, -1, -1, -1, - -1, -1, -1, 8, -1, -1, 11, -1, -1, 36, - 15, 16, 17, 18, 19, 20, 21, -1, -1, 46, - -1, -1, -1, -1, -1, -1, 53, -1, -1, -1, - -1, 36, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 46, -1, -1, -1, -1, -1, -1, 53, -1, - 8, 78, -1, 11, -1, -1, -1, 15, 16, 17, - 18, 19, 20, 21, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 78, -1, -1, -1, -1, 36, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 46, -1, - -1, -1, -1, -1, -1, 53, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 8, -1, -1, 11, -1, - -1, -1, 15, 16, 17, 18, 19, 20, 21, -1, - 78, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 36, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 46, -1, 172, -1, -1, -1, -1, - 53, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 188, -1, -1, -1, -1, 193, 172, -1, -1, - -1, -1, -1, -1, -1, 78, -1, -1, -1, -1, - -1, -1, -1, 188, -1, -1, -1, -1, 193, 216, - 217, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 231, -1, -1, -1, -1, -1, - -1, 216, 217, -1, 172, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 231, -1, -1, -1, - 188, -1, -1, -1, -1, 193, -1, -1, -1, -1, - 267, -1, -1, 270, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 284, 216, 217, - 287, -1, 267, -1, -1, 270, -1, -1, -1, 172, - -1, -1, -1, 231, -1, -1, -1, -1, -1, 284, - -1, -1, 287, -1, -1, 188, -1, -1, -1, -1, - 193, -1, -1, -1, -1, 8, -1, -1, 11, -1, - -1, -1, 15, 16, 17, 18, 19, 20, 21, 267, - -1, -1, 270, 216, 217, -1, -1, -1, -1, -1, - -1, -1, -1, 36, -1, -1, 284, -1, 231, 287, - -1, -1, -1, 46, -1, -1, -1, -1, -1, -1, - 53, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 380, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 267, 78, -1, 270, -1, -1, - -1, -1, -1, -1, -1, 380, -1, -1, -1, -1, - -1, 284, -1, -1, 287, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 481, 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 503, -1, -1, 22, 23, 24, 25, 26, + 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, + 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, + 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, + 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, + 77, -1, -1, -1, 81, 82, 83, 84, 85, 86, + -1, 88, 89, 90, -1, 92, 93, 94, 95, 96, + 97, -1, -1, 100, 101, 102, 103, 104, 105, 106, + 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, + 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, + 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, + 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, + 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, + -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, + -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, -1, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, -1, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + -1, -1, 219, -1, 221, 222, 223, 224, 225, 226, + -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, + 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, + 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, + 257, 258, 259, 260, 261, -1, 263, 264, 265, 266, + 267, -1, 269, 270, -1, 272, -1, 274, 275, 276, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, -1, 294, -1, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, -1, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, -1, 355, 356, + 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, + 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, -1, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, -1, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, -1, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, -1, 454, -1, 456, + 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, + 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + -1, 3, 4, 5, -1, -1, 8, 9, -1, -1, + -1, -1, -1, 15, 16, -1, 503, 19, 20, 21, + 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, + 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, + 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, + 52, -1, 54, 55, 56, 57, 58, 59, 60, 61, + 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, + 72, 73, 74, 75, 76, -1, 78, 79, 80, 81, + 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, + 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, 113, 114, 115, 116, 117, 118, 119, 120, 121, + 122, 123, 124, 125, 126, 127, 128, 129, 130, 131, + 132, 133, 134, 135, 136, 137, 138, 139, 140, 141, + 142, 143, 144, 145, 146, 147, -1, 149, 150, 151, + 152, 153, -1, 155, 156, 157, 158, 159, 160, 161, + 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, + 172, 173, 174, 175, 176, -1, -1, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, + 192, 193, 194, 195, 196, 197, 198, 199, 200, 201, + 202, 203, 204, -1, 206, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, 227, 228, 229, 230, 231, + 232, 233, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, + 272, 273, -1, 275, 276, 277, 278, 279, 280, 281, + 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, + -1, 293, 294, 295, -1, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, + 312, -1, 314, 315, 316, -1, 318, 319, 320, 321, + 322, 323, 324, 325, 326, 327, 328, 329, 330, 331, + 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + -1, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, + 422, -1, 424, -1, 426, 427, 428, 429, 430, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 443, 444, 445, 446, 447, 448, 449, 450, -1, + 452, 453, 454, 455, 456, 457, 458, 459, 460, 461, + 462, 463, 464, 465, 466, 467, 468, 469, 470, 471, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 380, -1, -1, -1, -1, -1, -1, -1, + -1, 483, 484, 485, 486, -1, 3, -1, 490, 491, + 492, 8, 494, 495, 496, 497, 498, 499, 15, 16, + -1, -1, 19, 20, 21, 22, 23, 24, 25, 26, + 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, + 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, + 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, + 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, + 77, -1, -1, -1, 81, 82, 83, 84, 85, 86, + -1, 88, 89, 90, -1, 92, 93, 94, 95, 96, + 97, -1, -1, 100, 101, 102, 103, 104, 105, 106, + 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, + 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, + 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, + 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, + 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, + -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, + -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, -1, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, -1, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + -1, -1, 219, -1, 221, 222, 223, 224, 225, 226, + -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, + 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, + 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, + 257, 258, 259, 260, 261, -1, 263, 264, 265, 266, + 267, -1, 269, 270, -1, 272, -1, 274, 275, 276, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, -1, 294, -1, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, -1, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, -1, 355, 356, + 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, + 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, -1, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, -1, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, -1, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, -1, 454, -1, 456, + 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, + 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + -1, -1, -1, 490, 491, 492, -1, 494, 495, 496, + 497, 498, 499, 8, -1, -1, 11, -1, -1, -1, + 15, 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 36, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 46, -1, 8, -1, -1, 11, -1, 53, -1, + 15, 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 484, -1, 172, - 487, 488, 489, -1, 491, 492, 493, 494, 495, 496, - -1, -1, -1, -1, 501, 188, -1, 380, -1, 484, - 193, -1, 487, 488, 489, -1, 491, 492, 493, 494, - 495, 496, -1, -1, -1, -1, 501, -1, -1, -1, - -1, -1, -1, 216, 217, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 231, -1, - -1, -1, -1, -1, -1, -1, 484, -1, -1, 487, - 488, 489, -1, 491, 492, 493, 494, 495, 496, -1, - -1, -1, -1, 501, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 267, -1, -1, 270, -1, -1, + -1, 36, -1, 78, -1, -1, -1, -1, -1, -1, + -1, 46, -1, 8, -1, -1, 11, -1, 53, -1, + 15, 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 284, -1, -1, 287, -1, -1, -1, -1, -1, - -1, 484, -1, -1, 487, 488, 489, -1, 491, 492, - 493, 494, 495, 496, -1, -1, -1, -1, 501, -1, - -1, -1, -1, -1, 8, -1, -1, 11, -1, -1, - -1, 15, 16, 17, 18, 19, 20, 21, -1, -1, - -1, -1, -1, -1, -1, -1, 8, -1, -1, 11, - -1, -1, 36, 15, 16, 17, 18, 19, 20, 21, - -1, -1, 46, -1, -1, -1, -1, -1, -1, 53, - -1, -1, -1, -1, 36, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 46, -1, -1, 380, -1, -1, - -1, 53, -1, 8, 78, -1, 11, -1, -1, -1, + -1, 36, -1, 78, -1, -1, -1, -1, -1, -1, + -1, 46, -1, 8, -1, -1, 11, -1, 53, -1, 15, 16, 17, 18, 19, 20, 21, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 78, -1, -1, -1, - -1, 36, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 46, -1, -1, -1, -1, -1, -1, 53, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 8, -1, - -1, 11, -1, -1, -1, 15, 16, 17, 18, 19, - 20, 21, -1, 78, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 36, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 46, -1, 172, -1, - -1, 484, -1, 53, 487, 488, 489, -1, 491, 492, - 493, 494, 495, 496, 188, -1, -1, -1, 501, 193, - 172, -1, -1, -1, -1, -1, -1, -1, 78, -1, - -1, -1, -1, -1, -1, -1, 188, -1, -1, -1, - -1, 193, 216, 217, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 231, -1, -1, - -1, -1, -1, -1, 216, 217, -1, 172, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 231, - -1, -1, -1, 188, -1, -1, -1, -1, 193, -1, - -1, -1, -1, 267, -1, -1, 270, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 284, 216, 217, 287, -1, 267, -1, -1, 270, -1, - -1, -1, 172, -1, -1, -1, 231, -1, -1, -1, - -1, -1, 284, -1, -1, 287, -1, -1, 188, -1, - -1, -1, -1, 193, -1, -1, -1, -1, 8, -1, - -1, 11, -1, -1, -1, 15, 16, 17, 18, 19, - 20, 21, 267, -1, -1, 270, 216, 217, -1, -1, - -1, -1, -1, -1, -1, -1, 36, -1, -1, 284, - -1, 231, 287, -1, -1, -1, 46, -1, -1, -1, - -1, -1, -1, 53, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 380, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 267, 78, -1, - 270, -1, -1, -1, -1, -1, -1, -1, 380, -1, - -1, -1, -1, -1, 284, -1, -1, 287, -1, -1, + -1, 36, -1, 78, -1, -1, -1, -1, -1, -1, + -1, 46, -1, 8, -1, -1, 11, 172, 53, -1, + 15, 16, 17, 18, 19, 20, 21, -1, -1, -1, + -1, -1, -1, -1, 189, -1, -1, -1, -1, 194, + -1, 36, -1, 78, -1, -1, -1, -1, -1, -1, + -1, 46, -1, -1, -1, -1, -1, 172, 53, -1, + -1, -1, 217, 218, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 189, -1, -1, 232, -1, 194, + -1, -1, -1, 78, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 172, -1, -1, + -1, -1, 217, 218, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 268, 189, -1, 271, 232, -1, 194, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 285, -1, -1, 288, -1, -1, -1, 172, -1, -1, + -1, -1, 217, 218, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 268, 189, -1, 271, 232, -1, 194, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 285, -1, -1, 288, -1, -1, -1, 172, -1, -1, + -1, -1, 217, 218, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 268, 189, -1, 271, 232, -1, 194, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 380, -1, -1, -1, -1, + 285, -1, -1, 288, -1, -1, -1, -1, -1, -1, + -1, -1, 217, 218, -1, -1, -1, 382, -1, -1, + -1, -1, -1, 268, -1, -1, 271, 232, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 285, -1, -1, 288, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 382, -1, -1, + -1, -1, -1, 268, -1, -1, 271, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 285, -1, -1, 288, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 382, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 484, -1, 172, 487, 488, 489, -1, 491, 492, 493, - 494, 495, 496, -1, -1, -1, -1, 501, 188, -1, - 380, -1, 484, 193, -1, 487, 488, 489, -1, 491, - 492, 493, 494, 495, 496, -1, -1, -1, -1, 501, - -1, -1, -1, -1, -1, -1, 216, 217, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 231, -1, -1, -1, -1, -1, -1, -1, 484, - -1, -1, 487, 488, 489, -1, 491, 492, 493, 494, - 495, 496, -1, -1, -1, -1, 501, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 267, -1, -1, - 270, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 284, -1, -1, 287, -1, -1, - -1, -1, -1, -1, 484, -1, -1, 487, 488, 489, - -1, 491, 492, 493, 494, 495, 496, -1, -1, -1, - -1, 501, -1, -1, -1, -1, -1, 8, -1, -1, + -1, -1, 487, -1, -1, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, -1, -1, 382, -1, 504, + -1, -1, -1, -1, -1, -1, -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, 17, 18, 19, 20, - 21, -1, -1, -1, -1, -1, -1, -1, -1, 8, - -1, -1, 11, -1, -1, 36, 15, 16, 17, 18, - 19, 20, 21, -1, -1, 46, -1, -1, -1, -1, - -1, -1, 53, -1, -1, -1, -1, 36, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 46, -1, -1, - 380, -1, -1, -1, 53, -1, 8, 78, -1, 11, - -1, -1, -1, 15, 16, 17, 18, 19, 20, 21, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 78, - -1, -1, -1, -1, 36, -1, -1, -1, -1, -1, + 21, -1, 487, -1, -1, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, 36, -1, 382, -1, 504, + -1, -1, -1, -1, -1, 46, -1, 8, -1, -1, + 11, -1, 53, -1, 15, 16, 17, 18, 19, 20, + 21, -1, 487, -1, -1, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, 36, -1, 78, -1, 504, + -1, -1, -1, -1, -1, 46, -1, 8, -1, -1, + 11, -1, 53, -1, 15, 16, 17, 18, 19, 20, + 21, -1, 487, -1, -1, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, 36, -1, 78, -1, 504, + -1, -1, -1, -1, -1, 46, -1, 8, -1, -1, + 11, -1, 53, -1, 15, 16, 17, 18, 19, 20, + 21, -1, 487, -1, -1, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, 36, -1, 78, -1, 504, + -1, -1, -1, -1, -1, 46, -1, 8, -1, -1, + 11, 172, 53, -1, 15, 16, 17, 18, 19, 20, + 21, -1, -1, -1, -1, -1, -1, -1, 189, -1, + -1, -1, -1, 194, -1, 36, -1, 78, -1, -1, + -1, -1, -1, -1, -1, 46, -1, -1, -1, -1, + -1, 172, 53, -1, -1, -1, 217, 218, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 189, -1, + -1, 232, -1, 194, -1, -1, -1, 78, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 172, -1, -1, -1, -1, 217, 218, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 268, 189, -1, + 271, 232, -1, 194, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 285, -1, -1, 288, -1, -1, + -1, 172, -1, -1, -1, -1, 217, 218, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 268, 189, -1, + 271, 232, -1, 194, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 285, -1, -1, 288, -1, -1, + -1, 172, -1, -1, -1, -1, 217, 218, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 268, 189, -1, + 271, 232, -1, 194, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 285, -1, -1, 288, -1, -1, + -1, -1, -1, -1, -1, -1, 217, 218, -1, -1, + -1, 382, -1, -1, -1, -1, -1, 268, -1, -1, + 271, 232, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 285, -1, -1, 288, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 382, -1, -1, -1, -1, -1, 268, -1, -1, + 271, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 285, -1, -1, 288, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 382, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 487, -1, -1, 490, + 491, 492, -1, 494, 495, 496, 497, 498, 499, -1, + -1, 382, -1, 504, -1, -1, -1, -1, -1, -1, + -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, + 17, 18, 19, 20, 21, -1, 487, -1, -1, 490, + 491, 492, -1, 494, 495, 496, 497, 498, 499, 36, + -1, 382, -1, 504, -1, -1, -1, -1, -1, 46, + -1, 8, -1, -1, 11, -1, 53, -1, 15, 16, + 17, 18, 19, 20, 21, -1, 487, -1, -1, 490, + 491, 492, -1, 494, 495, 496, 497, 498, 499, 36, + -1, 78, -1, 504, -1, -1, -1, -1, -1, 46, + -1, 8, -1, -1, 11, -1, 53, -1, 15, 16, + 17, 18, 19, 20, 21, -1, 487, -1, -1, 490, + 491, 492, -1, 494, 495, 496, 497, 498, 499, 36, + -1, 78, -1, 504, -1, -1, -1, -1, -1, 46, + -1, 8, -1, -1, 11, -1, 53, -1, 15, 16, + 17, 18, 19, 20, 21, -1, 487, -1, -1, 490, + 491, 492, -1, 494, 495, 496, 497, 498, 499, 36, + -1, 78, -1, 504, -1, -1, -1, -1, -1, 46, + -1, 8, -1, -1, 11, 172, 53, -1, 15, 16, + 17, 18, 19, 20, 21, -1, -1, -1, -1, -1, + -1, -1, 189, -1, -1, -1, -1, 194, -1, 36, + -1, 78, -1, -1, -1, -1, -1, -1, -1, 46, + -1, -1, -1, -1, -1, 172, 53, -1, -1, -1, + 217, 218, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 189, -1, -1, 232, -1, 194, -1, -1, + -1, 78, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 172, -1, -1, -1, -1, + 217, 218, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 268, 189, -1, 271, 232, -1, 194, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 285, -1, + -1, 288, -1, -1, -1, 172, -1, -1, -1, -1, + 217, 218, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 268, 189, -1, 271, 232, -1, 194, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 285, -1, + -1, 288, -1, -1, -1, 172, -1, -1, -1, -1, + 217, 218, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 268, 189, -1, 271, 232, -1, 194, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 285, -1, + -1, 288, -1, -1, -1, -1, -1, -1, -1, -1, + 217, 218, -1, -1, -1, 382, -1, -1, -1, -1, + -1, 268, -1, -1, 271, 232, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 285, -1, + -1, 288, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 382, -1, -1, -1, -1, + -1, 268, -1, -1, 271, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 285, -1, + -1, 288, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 382, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 487, -1, -1, 490, 491, 492, -1, 494, 495, 496, + 497, 498, 499, -1, -1, 382, -1, 504, -1, -1, + -1, -1, -1, -1, -1, 8, -1, -1, 11, -1, + -1, -1, 15, 16, 17, 18, 19, 20, 21, -1, + 487, -1, -1, 490, 491, 492, -1, 494, 495, 496, + 497, 498, 499, 36, -1, 382, -1, 504, -1, -1, + -1, -1, -1, 46, -1, -1, 8, -1, -1, 11, + 53, -1, -1, 15, 16, 17, 18, 19, 20, 21, + 487, -1, -1, 490, 491, 492, -1, 494, 495, 496, + 497, 498, 499, -1, 36, 78, -1, 504, -1, -1, -1, -1, -1, -1, 46, -1, -1, -1, -1, -1, -1, 53, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, - 17, 18, 19, 20, 21, -1, 78, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 36, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 46, - -1, 172, -1, -1, 484, -1, 53, 487, 488, 489, - -1, 491, 492, 493, 494, 495, 496, 188, -1, -1, - -1, 501, 193, 172, -1, -1, -1, -1, -1, -1, - -1, 78, -1, -1, -1, -1, -1, -1, -1, 188, - -1, -1, -1, -1, 193, 216, 217, -1, -1, -1, + 487, -1, -1, 490, 491, 492, -1, 494, 495, 496, + 497, 498, 499, -1, -1, 502, 78, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 231, -1, -1, -1, -1, -1, -1, 216, 217, -1, - 172, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 231, -1, -1, -1, 188, -1, -1, -1, - -1, 193, -1, -1, -1, -1, 267, -1, -1, 270, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 284, 216, 217, 287, -1, 267, -1, - -1, 270, -1, -1, -1, 172, -1, -1, -1, 231, - -1, -1, -1, -1, -1, 284, -1, -1, 287, -1, - -1, 188, -1, -1, -1, -1, 193, -1, -1, -1, + 487, -1, -1, 490, 491, 492, -1, 494, 495, 496, + 497, 498, 499, -1, -1, 502, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 172, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 267, -1, -1, 270, 216, - 217, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 284, -1, 231, 287, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 189, -1, -1, -1, + -1, 194, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 380, + 172, -1, -1, -1, 217, 218, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 189, -1, 232, + -1, -1, 194, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 267, -1, -1, 270, -1, -1, -1, -1, -1, -1, - -1, 380, -1, -1, -1, -1, -1, 284, -1, -1, - 287, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 217, 218, -1, -1, -1, + -1, -1, -1, -1, -1, 268, -1, -1, 271, -1, + 232, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 285, -1, -1, 288, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 268, -1, -1, 271, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 380, -1, + -1, -1, -1, 285, -1, -1, 288, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 484, -1, -1, 487, 488, 489, -1, - 491, 492, 493, 494, 495, 496, -1, -1, -1, -1, - 501, -1, -1, 380, -1, 484, -1, -1, 487, 488, - 489, -1, 491, 492, 493, 494, 495, 496, -1, -1, - 499, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 382, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 484, -1, -1, 487, 488, 489, -1, 491, - 492, 493, 494, 495, 496, -1, -1, 499, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3, -1, -1, 484, -1, -1, - 487, 488, 489, -1, 491, 492, 493, 494, 495, 496, - -1, -1, 499, 22, 23, 24, 25, 26, 27, 28, - 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, - 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, - 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, - 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, - 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, - 79, 80, 81, 82, 83, 84, 85, 86, 87, 88, - 89, 90, 91, 92, 93, 94, 95, 96, 97, 98, - 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, 117, 118, - 119, 120, 121, 122, 123, 124, 125, 126, 127, 128, - 129, 130, 131, 132, 133, 134, 135, 136, 137, 138, - 139, 140, 141, 142, 143, 144, 145, 146, 147, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - 159, 160, 161, 162, 163, 164, 165, 166, 167, 168, - 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, - 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, - 199, 200, 201, 202, 203, 204, 205, 206, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, 217, 218, - 219, 220, 221, 222, 223, 224, 225, 226, 227, 228, - 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, - 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, - 279, 280, 281, 282, 283, 284, 285, 286, 287, 288, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, - 319, 320, 321, 322, 323, 324, 325, 326, 327, 328, - 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 436, 437, 438, - 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, - 449, 450, 451, 452, 453, 454, 455, 456, 457, 458, - 459, 460, 461, 462, 463, 464, 465, 466, 467, 468, - 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 3, -1, -1, -1, -1, - -1, -1, -1, -1, 493, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, - 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, - 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, - -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, - 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, - -1, -1, -1, 81, 82, 83, 84, 85, 86, -1, - 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, - -1, -1, 100, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, - 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, - -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, - 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, - 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, - -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - -1, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, -1, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, -1, -1, - 218, -1, 220, 221, 222, 223, 224, 225, -1, -1, - 228, -1, 230, -1, -1, 233, 234, 235, 236, 237, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, -1, 262, 263, 264, 265, 266, -1, - 268, 269, -1, 271, -1, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, -1, 292, -1, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, -1, -1, -1, - -1, 309, 310, 311, -1, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, -1, 353, 354, 355, 356, 357, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - -1, -1, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, -1, 381, 382, 383, 384, -1, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, -1, 406, -1, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, -1, 424, 425, 426, 427, - 428, 429, 430, 431, -1, -1, 434, 435, 436, -1, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, -1, 453, 454, 455, 456, 457, - -1, -1, 460, -1, -1, 463, 464, 465, 466, 467, - 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 3, -1, -1, -1, - -1, -1, -1, -1, -1, 493, -1, -1, -1, -1, + 382, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 487, -1, -1, 490, 491, 492, + -1, 494, 495, 496, 497, 498, 499, -1, -1, 502, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 3, -1, -1, 487, -1, -1, 490, 491, + 492, -1, 494, 495, 496, 497, 498, 499, -1, -1, + 502, 22, 23, 24, 25, 26, 27, 28, 29, 30, + 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, + 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, + 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, + 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, + 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, + 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, + 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, + 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, 115, 116, 117, 118, 119, 120, + 121, 122, 123, 124, 125, 126, 127, 128, 129, 130, + 131, 132, 133, 134, 135, 136, 137, 138, 139, 140, + 141, 142, 143, 144, 145, 146, 147, 148, 149, 150, + 151, 152, 153, 154, 155, 156, 157, 158, 159, 160, + 161, 162, 163, 164, 165, 166, 167, 168, 169, 170, + 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, + 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, + 191, 192, 193, 194, 195, 196, 197, 198, 199, 200, + 201, 202, 203, 204, 205, 206, 207, 208, 209, 210, + 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, + 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, + 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 321, 322, 323, 324, 325, 326, 327, 328, 329, 330, + 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 431, 432, 433, 434, 435, 436, 437, 438, 439, 440, + 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, + 451, 452, 453, 454, 455, 456, 457, 458, 459, 460, + 461, 462, 463, 464, 465, 466, 467, 468, 469, 470, + 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 3, -1, -1, -1, + -1, -1, -1, -1, -1, 496, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, @@ -15136,86 +15204,135 @@ static const yytype_int16 yycheck[] = 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, -1, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, -1, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, -1, - -1, 218, -1, 220, 221, 222, 223, 224, 225, -1, - -1, 228, -1, 230, -1, -1, 233, 234, 235, 236, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, -1, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, -1, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + -1, -1, 219, -1, 221, 222, 223, 224, 225, 226, + -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, -1, 262, 263, 264, 265, 266, - -1, 268, 269, -1, 271, -1, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, -1, 292, -1, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, -1, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, -1, 353, 354, 355, 356, + 257, 258, 259, 260, 261, -1, 263, 264, 265, 266, + 267, -1, 269, 270, -1, 272, -1, 274, 275, 276, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, -1, 294, -1, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, -1, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, -1, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - -1, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, -1, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, -1, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, + 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, -1, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, -1, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, -1, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, -1, 454, -1, 456, + 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 3, 4, 5, - -1, -1, -1, 9, -1, -1, 493, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, - 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, - -1, 37, -1, -1, -1, -1, 42, 43, 44, -1, - 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, - 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, - -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, - 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, - 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, - 96, 97, -1, 99, 100, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, -1, - 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, - 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, - 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, - 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, 203, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, - 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, - -1, -1, 228, 229, 230, 231, -1, 233, 234, 235, - 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, - 266, -1, 268, 269, 270, 271, -1, 273, 274, 275, - 276, 277, 278, -1, 280, 281, 282, -1, 284, 285, - 286, -1, -1, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, 312, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, 352, 353, 354, 355, - 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, 407, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, 452, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, - 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 3, 4, - 5, -1, -1, -1, 9, 491, 492, -1, -1, -1, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 3, -1, -1, -1, -1, -1, -1, -1, -1, 496, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, + 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, + 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, + 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, + 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, + 73, 74, 75, 76, 77, -1, -1, -1, 81, 82, + 83, 84, 85, 86, -1, 88, 89, 90, -1, 92, + 93, 94, 95, 96, 97, -1, -1, 100, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, + 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, + 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, + 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, + 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, + 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, + 183, 184, 185, 186, 187, 188, -1, 190, 191, 192, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, -1, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, -1, -1, 219, -1, 221, 222, + 223, 224, 225, 226, -1, -1, 229, -1, 231, -1, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, + 253, 254, 255, 256, 257, 258, 259, 260, 261, -1, + 263, 264, 265, 266, 267, -1, 269, 270, -1, 272, + -1, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, + -1, 294, -1, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, -1, -1, -1, -1, 311, 312, + 313, -1, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, + 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, -1, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, -1, 408, -1, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, -1, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, -1, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, -1, 456, 457, 458, 459, 460, -1, -1, + 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, + 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 3, 4, 5, -1, -1, -1, + 9, -1, -1, 496, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, + 29, -1, 31, 32, 33, -1, -1, -1, 37, -1, + -1, -1, -1, 42, 43, 44, -1, 46, 47, 48, + 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, + 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, + 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, + 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, + 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, + 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, + 109, 110, 111, 112, 113, 114, -1, 116, 117, 118, + 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, + -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, + -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, + 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, + 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, + 169, 170, 171, 172, 173, -1, 175, -1, 177, 178, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, 204, 205, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, -1, -1, + 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, + 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, + 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, + 259, 260, 261, 262, 263, 264, 265, 266, 267, -1, + 269, 270, 271, 272, -1, 274, 275, 276, 277, 278, + 279, -1, 281, 282, 283, -1, 285, 286, 287, -1, + -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, + 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 384, 385, 386, -1, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, + 399, 400, 401, 402, 403, -1, 405, 406, -1, 408, + 409, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, 423, 424, 425, 426, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, 451, 452, -1, 454, 455, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, + 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, + 479, 480, 481, 482, 483, 484, 485, 486, 3, 4, + 5, -1, -1, -1, 9, 494, 495, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, 37, -1, -1, -1, -1, 42, 43, 44, @@ -15232,265 +15349,303 @@ static const yytype_int16 yycheck[] = 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, 203, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, - 225, -1, -1, 228, 229, 230, 231, -1, 233, 234, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, 204, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, - 265, 266, -1, 268, 269, 270, 271, -1, 273, 274, - 275, 276, 277, 278, -1, 280, 281, 282, -1, 284, - 285, 286, -1, -1, 289, 290, 291, 292, 293, 294, + 265, 266, 267, -1, 269, 270, 271, 272, -1, 274, + 275, 276, 277, 278, 279, -1, 281, 282, 283, -1, + 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, 312, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, 352, 353, 354, + 305, 306, -1, -1, -1, -1, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, 369, 370, 371, 372, 373, 374, + 365, 366, 367, 368, 369, -1, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, 407, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, 452, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 3, - 4, 5, -1, -1, -1, 9, 491, 492, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, - 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, - -1, -1, -1, 37, -1, -1, -1, -1, 42, 43, - 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, - 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, - 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, - 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, - 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, - 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, - 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, - 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, - 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, - 154, 155, 156, -1, 158, 159, 160, 161, -1, 163, - -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, 188, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, 203, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, 216, 217, 218, 219, 220, 221, 222, 223, - 224, 225, -1, -1, 228, 229, 230, 231, -1, 233, - 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, - 264, 265, 266, -1, 268, 269, 270, 271, -1, 273, - 274, 275, 276, 277, 278, -1, 280, 281, 282, -1, - 284, 285, 286, -1, -1, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, 312, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, 352, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, 407, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, 452, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - -1, -1, 8, -1, -1, 11, -1, 491, 492, 15, - 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 36, -1, -1, -1, -1, 41, -1, -1, -1, -1, - 46, -1, 8, -1, -1, 11, -1, 53, -1, 15, - 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 36, -1, 78, -1, -1, -1, -1, -1, -1, -1, - 46, -1, 8, -1, -1, 11, -1, 53, -1, 15, - 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, 409, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + 455, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 3, 4, 5, -1, -1, -1, 9, 494, + 495, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, + 31, 32, 33, -1, -1, -1, 37, -1, -1, -1, + -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, + 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, + 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, + 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, + 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, + -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, + 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, + 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, + 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, + 151, 152, 153, 154, 155, 156, -1, 158, 159, 160, + 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, + 171, 172, 173, -1, 175, -1, 177, 178, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, 204, 205, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, -1, -1, 229, 230, + 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, + 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, + 261, 262, 263, 264, 265, 266, 267, -1, 269, 270, + 271, 272, -1, 274, 275, 276, 277, 278, 279, -1, + 281, 282, 283, -1, 285, 286, 287, -1, -1, 290, + 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 402, 403, -1, 405, 406, -1, 408, 409, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, + 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, + 431, 432, 433, 434, -1, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + 451, 452, -1, 454, 455, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, + 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, -1, -1, 8, -1, + -1, 11, -1, 494, 495, 15, 16, 17, 18, 19, + 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 36, -1, -1, -1, + -1, 41, -1, -1, -1, -1, 46, -1, 8, -1, + -1, 11, -1, 53, -1, 15, 16, 17, 18, 19, + 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 36, -1, 78, -1, + -1, -1, -1, -1, -1, -1, 46, -1, 8, -1, + -1, 11, -1, 53, -1, 15, 16, 17, 18, 19, + 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 36, -1, 78, -1, + -1, -1, 122, -1, -1, -1, 46, -1, 8, -1, + -1, 11, -1, 53, -1, 15, 16, 17, 18, 19, + 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 36, -1, 78, -1, + 40, -1, -1, -1, -1, -1, 46, -1, -1, -1, + -1, -1, 172, 53, -1, 8, -1, -1, 11, -1, + -1, -1, 15, 16, 17, 18, 19, 20, 21, 189, + -1, -1, -1, -1, 194, -1, -1, -1, 78, -1, + -1, -1, 162, 36, -1, -1, -1, 167, -1, -1, + -1, -1, 172, 46, -1, -1, -1, 217, 218, -1, + 53, -1, -1, -1, -1, -1, -1, -1, -1, 189, + -1, -1, 232, -1, 194, -1, -1, -1, -1, -1, + -1, 161, -1, -1, -1, 78, -1, -1, -1, -1, + -1, -1, 172, -1, -1, -1, -1, 217, 218, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 268, 189, + -1, 271, 232, -1, 194, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 285, -1, -1, 288, -1, + -1, -1, 172, -1, -1, -1, -1, 217, 218, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 268, 189, + -1, 271, 232, -1, 194, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 285, -1, -1, 288, -1, + -1, -1, -1, -1, -1, -1, -1, 217, 218, 172, + -1, -1, -1, -1, -1, -1, -1, -1, 268, -1, + -1, 271, 232, -1, -1, -1, 189, -1, -1, -1, + -1, 194, -1, -1, -1, 285, -1, -1, 288, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 36, -1, 78, -1, -1, -1, 122, -1, -1, -1, - 46, -1, 8, -1, -1, 11, -1, 53, -1, 15, - 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, + -1, -1, 382, -1, 217, 218, -1, -1, 268, -1, + -1, 271, -1, -1, -1, -1, 316, -1, -1, 232, + -1, -1, -1, -1, -1, 285, -1, -1, 288, -1, + -1, -1, -1, -1, -1, -1, 8, -1, -1, 11, + -1, -1, 382, 15, 16, 17, 18, 19, 20, 21, + -1, -1, -1, -1, -1, 268, -1, -1, 271, -1, + -1, -1, -1, -1, 36, 445, -1, -1, 40, -1, + -1, -1, 285, -1, 46, 288, -1, -1, -1, -1, + -1, 53, 382, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 310, -1, -1, + -1, -1, -1, -1, -1, -1, 78, 487, -1, -1, + 490, 491, 492, -1, 494, 495, 496, 497, 498, 499, + -1, -1, 382, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 36, -1, 78, -1, 40, -1, -1, -1, -1, -1, - 46, -1, 8, -1, -1, 11, 172, 53, -1, 15, - 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, - -1, -1, 188, -1, -1, -1, -1, 193, -1, -1, - 36, -1, 78, -1, -1, -1, 162, -1, -1, -1, - 46, 167, -1, -1, -1, -1, 172, 53, -1, -1, - 216, 217, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 188, -1, -1, 231, -1, 193, -1, -1, - -1, -1, 78, -1, -1, 161, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 172, -1, -1, -1, - 216, 217, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 267, 188, -1, 270, 231, -1, 193, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 284, -1, - -1, 287, -1, -1, -1, -1, 172, -1, -1, -1, - 216, 217, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 267, 188, -1, 270, 231, -1, 193, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 284, -1, - -1, 287, -1, -1, -1, -1, 172, -1, -1, -1, - 216, 217, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 267, 188, -1, 270, 231, -1, 193, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 284, -1, - -1, 287, -1, -1, -1, -1, -1, -1, -1, -1, - 216, 217, -1, -1, 380, -1, -1, -1, -1, -1, - -1, 267, -1, -1, 270, 231, -1, -1, 314, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 284, -1, - -1, 287, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 380, -1, -1, -1, -1, -1, - -1, 267, -1, -1, 270, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 442, -1, 284, -1, - -1, 287, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 380, -1, -1, -1, -1, -1, - -1, -1, 308, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 484, -1, - -1, 487, 488, 489, -1, 491, 492, 493, 494, 495, - 496, -1, -1, -1, 380, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 487, -1, -1, + 490, 491, 492, -1, 494, 495, 496, 497, 498, 499, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 382, -1, -1, -1, 8, -1, -1, 11, -1, -1, -1, - 15, 16, 17, 18, 19, 20, 21, -1, 484, -1, - -1, 487, 488, 489, -1, 491, 492, 493, 494, 495, - 496, 36, -1, -1, 380, 40, -1, -1, -1, -1, - -1, 46, -1, 8, -1, -1, 11, -1, 53, -1, - 15, 16, 17, 18, 19, 20, 21, -1, 484, -1, - -1, 487, 488, 489, -1, 491, 492, 493, 494, 495, - 496, 36, -1, 78, -1, -1, -1, -1, -1, -1, - -1, 46, -1, 8, -1, -1, 11, -1, 53, -1, - 15, 16, 17, 18, 19, 20, 21, -1, 484, -1, - -1, 487, 488, 489, -1, 491, 492, 493, 494, 495, - 496, 36, -1, 78, -1, 40, -1, -1, -1, -1, - -1, 46, -1, 8, -1, -1, 11, -1, 53, -1, - 15, 16, 17, 18, 19, 20, 21, -1, 484, -1, - -1, 487, 488, 489, -1, 491, 492, 493, 494, 495, - 496, 36, -1, 78, -1, -1, -1, -1, -1, -1, - -1, 46, -1, -1, -1, -1, -1, 172, 53, -1, + 15, 16, 17, 18, 19, 20, 21, 487, -1, -1, + 490, 491, 492, -1, 494, 495, 496, 497, 498, 499, + 172, 36, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 46, -1, -1, -1, -1, -1, 189, 53, -1, + -1, -1, 194, -1, -1, -1, -1, 487, -1, -1, + 490, 491, 492, -1, 494, 495, 496, 497, 498, 499, + -1, -1, -1, 78, -1, 217, 218, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 188, -1, -1, -1, -1, 193, -1, - -1, -1, -1, 78, -1, -1, -1, -1, -1, -1, - -1, -1, 167, -1, -1, -1, -1, 172, -1, -1, - -1, 216, 217, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 188, -1, -1, 231, -1, 193, -1, + 232, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 487, -1, -1, 490, 491, 492, + -1, 494, 495, 496, 497, 498, 499, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 268, 8, -1, 271, + 11, -1, -1, -1, 15, 16, 17, 18, 19, 20, + 21, -1, -1, 285, -1, -1, 288, -1, -1, -1, + -1, -1, -1, -1, -1, 36, -1, -1, -1, 40, + -1, -1, 167, -1, -1, 46, -1, 172, -1, -1, + -1, -1, 53, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 189, -1, -1, -1, -1, 194, + -1, -1, -1, -1, -1, -1, -1, 78, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 172, -1, -1, - -1, 216, 217, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 267, 188, -1, 270, 231, -1, 193, -1, - -1, -1, -1, -1, -1, -1, -1, 162, -1, 284, - -1, -1, 287, -1, -1, -1, -1, 172, -1, -1, - -1, 216, 217, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 267, 188, -1, 270, 231, -1, 193, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 284, - -1, -1, 287, -1, -1, -1, -1, -1, -1, -1, - -1, 216, 217, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 267, -1, -1, 270, 231, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 284, - -1, -1, 287, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 380, -1, -1, -1, -1, - -1, -1, 267, -1, -1, 270, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 284, - -1, -1, 287, -1, -1, -1, -1, -1, -1, -1, - -1, 8, -1, -1, 11, 380, -1, -1, 15, 16, - 17, 18, 19, 20, 21, -1, -1, -1, -1, -1, - -1, -1, -1, 8, -1, -1, 11, -1, -1, 36, - 15, 16, 17, 18, 19, 20, 21, -1, -1, 46, - -1, -1, -1, -1, -1, 380, 53, -1, -1, -1, - -1, 36, -1, -1, -1, 40, -1, -1, -1, -1, - -1, 46, -1, -1, -1, -1, -1, -1, 53, 484, - -1, 78, 487, 488, 489, -1, 491, 492, 493, 494, - 495, 496, -1, -1, -1, 380, -1, -1, -1, -1, - -1, -1, -1, 78, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 484, - -1, -1, 487, 488, 489, -1, 491, 492, 493, 494, - 495, 496, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 217, 218, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 232, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 484, - -1, -1, 487, 488, 489, -1, 491, 492, 493, 494, - 495, 496, -1, -1, -1, 172, -1, 8, -1, -1, + 382, -1, -1, -1, -1, -1, -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, 17, 18, 19, 20, - 21, 188, -1, -1, -1, -1, 193, 172, -1, 484, - -1, -1, 487, 488, 489, 36, 491, 492, 493, 494, - 495, 496, -1, 188, -1, 46, -1, -1, 193, 216, - 217, -1, 53, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 231, -1, -1, -1, -1, -1, - -1, 216, 217, -1, -1, -1, -1, 78, -1, -1, - -1, -1, -1, -1, -1, -1, 231, -1, -1, -1, + 21, -1, -1, 268, -1, -1, 271, -1, -1, -1, + -1, -1, -1, -1, -1, 36, -1, -1, -1, -1, + 285, -1, -1, 288, -1, 46, -1, -1, -1, -1, + -1, 172, 53, -1, 8, -1, -1, 11, -1, -1, + -1, 15, 16, 17, 18, 19, 20, 21, 189, -1, + -1, -1, -1, 194, -1, -1, -1, 78, -1, -1, + -1, -1, 36, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 46, -1, -1, -1, 217, 218, -1, 53, + -1, -1, -1, -1, -1, 487, -1, -1, 490, 491, + 492, 232, 494, 495, 496, 497, 498, 499, -1, -1, + -1, -1, -1, -1, 78, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 382, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 268, -1, -1, + 271, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 162, -1, -1, 285, -1, -1, 288, -1, -1, + -1, 172, -1, -1, 8, -1, -1, 11, -1, -1, + -1, 15, 16, 17, 18, 19, 20, 21, 189, -1, + -1, -1, -1, 194, -1, -1, -1, -1, -1, -1, + -1, -1, 36, -1, -1, -1, 40, -1, -1, -1, + -1, -1, 46, -1, -1, -1, 217, 218, 172, 53, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 232, -1, -1, -1, 189, -1, -1, -1, -1, + 194, -1, 487, -1, 78, 490, 491, 492, -1, 494, + 495, 496, 497, 498, 499, -1, -1, -1, -1, -1, + -1, 382, -1, 217, 218, -1, -1, 268, -1, -1, + 271, -1, -1, -1, -1, -1, -1, -1, 232, -1, + -1, -1, -1, -1, 285, -1, -1, 288, -1, -1, + -1, -1, 8, -1, -1, 11, -1, -1, -1, 15, + 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, + -1, -1, -1, -1, 268, -1, -1, 271, -1, -1, + 36, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 46, 285, -1, -1, 288, -1, -1, 53, 172, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 189, -1, -1, -1, -1, + 194, -1, 78, -1, -1, -1, 487, -1, -1, 490, + 491, 492, -1, 494, 495, 496, 497, 498, 499, -1, + -1, 382, -1, 217, 218, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 232, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 267, -1, -1, 270, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 284, -1, -1, - 287, -1, 267, -1, -1, 270, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 284, - -1, -1, 287, -1, -1, -1, -1, -1, -1, -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, 17, - 18, 19, 20, 21, -1, -1, -1, -1, -1, -1, - -1, 172, 8, -1, -1, 11, -1, -1, 36, 15, - 16, -1, -1, 19, 20, 21, -1, 188, 46, -1, - -1, 8, 193, -1, 11, 53, -1, -1, 15, 16, - 36, -1, 19, 20, 21, -1, -1, -1, -1, -1, - 46, -1, -1, 380, -1, 216, 217, 53, -1, -1, - 78, -1, -1, -1, -1, -1, -1, -1, -1, 46, - 231, -1, -1, -1, -1, 380, 53, 8, -1, -1, - 11, -1, 78, -1, 15, 16, 413, -1, 19, 20, - 21, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 78, -1, -1, -1, -1, 267, -1, -1, 270, - -1, -1, -1, -1, -1, 46, -1, -1, -1, -1, - -1, -1, 53, 284, -1, -1, 287, -1, -1, -1, + 18, 19, 20, 21, -1, -1, -1, -1, 382, -1, + -1, -1, -1, -1, 268, -1, -1, 271, 36, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 46, -1, + -1, 285, -1, -1, 288, 53, 172, -1, -1, -1, + -1, 415, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 189, -1, -1, -1, -1, 194, -1, + 78, -1, -1, -1, -1, -1, 487, -1, -1, 490, + 491, 492, -1, 494, 495, 496, 497, 498, 499, -1, + -1, 217, 218, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 232, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 487, -1, -1, 490, 491, 492, -1, + 494, 495, 496, 497, 498, 499, -1, -1, 382, -1, + -1, -1, 268, -1, -1, 271, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 285, + -1, -1, 288, -1, 172, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 189, -1, -1, -1, -1, 194, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 217, + 218, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 232, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 487, -1, -1, 490, 491, 492, -1, + 494, 495, 496, 497, 498, 499, 382, -1, -1, -1, + 268, -1, -1, 271, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 285, -1, -1, + 288, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 78, -1, -1, - -1, -1, -1, -1, 172, -1, -1, 484, -1, -1, - 487, 488, 489, -1, 491, 492, 493, 494, 495, 496, - 188, -1, -1, -1, -1, 193, 172, -1, -1, 484, - -1, -1, 487, 488, 489, -1, 491, 492, 493, 494, - 495, 496, 188, -1, -1, 172, -1, 193, 216, 217, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 188, -1, 231, -1, -1, 193, -1, -1, 380, - 216, 217, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 231, -1, -1, -1, 216, - 217, 172, -1, -1, -1, -1, -1, -1, -1, 267, - -1, -1, 270, -1, 231, -1, -1, 188, -1, -1, - -1, -1, 193, -1, -1, -1, 284, -1, -1, 287, - -1, 267, -1, -1, 270, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 216, 217, -1, 284, -1, - 267, -1, -1, 270, -1, -1, -1, -1, -1, -1, - 231, -1, -1, -1, -1, -1, -1, 284, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 484, -1, -1, 487, 488, 489, -1, - 491, 492, 493, 494, 495, 496, -1, -1, -1, 270, + -1, 487, -1, -1, 490, 491, 492, -1, 494, 495, + 496, 497, 498, 499, 382, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 284, -1, -1, -1, -1, -1, -1, - -1, -1, 380, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 380, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 380, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 380, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 484, -1, -1, 487, - 488, 489, -1, 491, 492, 493, 494, 495, 496, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 484, -1, - -1, 487, 488, 489, -1, 491, 492, 493, 494, 495, - 496, -1, -1, -1, -1, -1, -1, 484, -1, -1, - 487, 488, 489, -1, 491, 492, 493, 494, 495, 496, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 3, -1, 5, - -1, -1, -1, 484, -1, -1, 487, 488, 489, -1, - 491, 492, 493, 494, 495, 496, 22, 23, 24, 25, + -1, -1, -1, 3, -1, 5, -1, -1, -1, 487, + -1, -1, 490, 491, 492, -1, 494, 495, 496, 497, + 498, 499, 22, 23, 24, 25, 26, 27, 28, 29, + 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, + 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, + 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, + 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, + 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, + 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, + 90, 91, 92, 93, 94, 95, 96, 97, 98, 99, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, 113, 114, 115, 116, 117, 118, 119, + 120, 121, 122, 123, 124, 125, 126, 127, 128, 129, + 130, 131, 132, 133, 134, 135, 136, 137, 138, 139, + 140, 141, 142, 143, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, + 160, 161, 162, 163, 164, 165, 166, 167, 168, 169, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 194, 195, 196, 197, 198, 199, + 200, 201, 202, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, 213, 214, 215, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, + 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, + 280, 281, 282, 283, 284, 285, 286, 287, 288, 289, + 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, + 310, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, 321, 322, 323, 324, 325, 326, 327, 328, 329, + 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, + 430, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, + 450, 451, 452, 453, 454, 455, 456, 457, 458, 459, + 460, 461, 462, 463, 464, 465, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 3, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, @@ -15500,290 +15655,195 @@ static const yytype_int16 yycheck[] = 86, 87, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, - 126, 127, 128, 129, 130, 131, 132, 133, 134, 135, - 136, 137, 138, 139, 140, 141, 142, 143, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, 159, 160, 161, 162, 163, 164, 165, - 166, 167, 168, 169, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, - 196, 197, 198, 199, 200, 201, 202, 203, 204, 205, - 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, - 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, - 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, - 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, - 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, - 316, 317, 318, 319, 320, 321, 322, 323, 324, 325, - 326, 327, 328, 329, 330, 331, 332, 333, 334, 335, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, - 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, - 456, 457, 458, 459, 460, 461, 462, 463, 464, 465, - 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 3, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, - 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, - 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, - 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, - 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, - 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, - 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, - 95, 96, 97, 98, 99, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, - 125, 126, 127, 128, 129, 130, 131, 132, 133, 134, - 135, 136, 137, 138, 139, 140, 141, 142, 143, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, 159, 160, 161, 162, 163, 164, - 165, 166, 167, 168, 169, 170, 171, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, 193, 194, - 195, 196, 197, 198, 199, 200, 201, 202, 203, 204, - 205, 206, 207, 208, 209, 210, 211, 212, 213, 214, - 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, - 225, 226, 227, 228, 229, 230, 231, 232, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, - 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, - 275, 276, 277, 278, 279, 280, 281, 282, 283, 284, - 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 315, 316, 317, 318, 319, 320, 321, 322, 323, 324, - 325, 326, 327, 328, 329, 330, 331, 332, 333, 334, - 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, - 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, - 455, 456, 457, 458, 459, 460, 461, 462, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 3, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, - 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, - 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, - 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, - 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, - 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, - 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, - 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, - 94, 95, 96, 97, 98, 99, 100, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, - 124, 125, 126, 127, 128, 129, 130, 131, 132, 133, - 134, 135, 136, 137, 138, 139, 140, 141, 142, 143, - 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, 159, 160, 161, 162, 163, - 164, 165, 166, 167, 168, 169, 170, 171, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, - 194, 195, 196, 197, 198, 199, 200, 201, 202, 203, - 204, 205, 206, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, - 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 274, 275, 276, 277, 278, 279, 280, 281, 282, 283, - 284, 285, 286, 287, 288, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 315, 316, 317, 318, 319, 320, 321, 322, 323, - 324, 325, 326, 327, 328, 329, 330, 331, 332, 333, - 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, - 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, - 454, 455, 456, 457, 458, 459, 460, 461, 462, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, - 33, -1, -1, -1, 37, -1, -1, -1, -1, 42, - 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, - 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, - 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, - 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, 99, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, 155, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - 203, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, 216, 217, 218, 219, 220, 221, 222, - 223, 224, 225, -1, -1, 228, 229, 230, 231, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, - 263, 264, 265, 266, -1, 268, 269, 270, 271, -1, - 273, 274, 275, 276, 277, 278, -1, 280, 281, 282, - -1, 284, 285, 286, -1, -1, 289, 290, 291, 292, - 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, 312, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, 352, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, 369, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, 407, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - 423, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, 452, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 3, -1, -1, -1, -1, -1, -1, -1, -1, + 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, + 126, 127, 128, 129, 130, 131, 132, 133, 134, 135, + 136, 137, 138, 139, 140, 141, 142, 143, 144, 145, + 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, 159, 160, 161, 162, 163, 164, 165, + 166, 167, 168, 169, 170, 171, 172, 173, 174, 175, + 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, + 196, 197, 198, 199, 200, 201, 202, 203, 204, 205, + 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, + 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, + 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 321, 322, 323, 324, 325, + 326, 327, 328, 329, 330, 331, 332, 333, 334, 335, + 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, + 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, + 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, + 456, 457, 458, 459, 460, 461, 462, 463, 464, 465, + 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, - 32, 33, 34, 35, -1, 37, -1, -1, -1, -1, - 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, - 52, 53, 54, 55, 56, 57, -1, 59, 60, 61, - 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, - -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, - 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, - 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, + 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, + 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, + 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, + 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, + 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, + 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, + 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, + 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, - -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, - 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, - 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, - -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, - 172, 173, -1, 175, -1, 177, 178, 179, -1, 181, + 112, 113, 114, 115, 116, 117, 118, 119, 120, 121, + 122, 123, 124, 125, 126, 127, 128, 129, 130, 131, + 132, 133, 134, 135, 136, 137, 138, 139, 140, 141, + 142, 143, 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, 155, 156, 157, 158, 159, 160, 161, + 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, + 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, + 192, 193, 194, 195, 196, 197, 198, 199, 200, 201, 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, -1, -1, 228, 229, 230, 231, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, + 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, 227, 228, 229, 230, 231, + 232, 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 264, 265, 266, -1, 268, 269, 270, 271, - -1, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, 291, + 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, + 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, + 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - 312, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, + 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, + 312, 313, 314, 315, 316, 317, 318, 319, 320, 321, + 322, 323, 324, 325, 326, 327, 328, 329, 330, 331, + 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, 369, 370, 371, + 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, + 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, 407, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, -1, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - 452, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, + 452, 453, 454, 455, 456, 457, 458, 459, 460, 461, + 462, 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 3, -1, -1, -1, -1, -1, -1, -1, + 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, - -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, - 51, 52, 53, 54, 55, 56, 57, -1, 59, 60, - 61, 62, 63, 64, -1, 66, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, - 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, - 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, - 171, 172, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, 203, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, 216, 217, 218, 219, 220, - 221, 222, 223, 224, 225, -1, -1, 228, 229, 230, - 231, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - 261, 262, 263, 264, 265, 266, -1, 268, 269, 270, - 271, -1, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, 312, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, -1, 406, 407, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, -1, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, 452, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, - 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 3, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, + 28, 29, -1, 31, 32, 33, -1, -1, -1, 37, + -1, -1, -1, -1, 42, 43, 44, -1, 46, 47, + 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, + -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, + 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, + -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, + 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, + -1, 99, 100, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, + 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, + -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, + 148, 149, 150, 151, 152, 153, 154, 155, 156, -1, + 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, + 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, 204, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, 217, + 218, 219, 220, 221, 222, 223, 224, 225, 226, -1, + -1, 229, 230, 231, 232, -1, 234, 235, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, + -1, 269, 270, 271, 272, -1, 274, 275, 276, 277, + 278, 279, -1, 281, 282, 283, -1, 285, 286, 287, + -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + 348, 349, 350, -1, 352, 353, 354, 355, 356, 357, + 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, -1, 371, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, 382, 383, 384, 385, 386, -1, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, 409, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, 426, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, 455, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + 34, 35, -1, 37, -1, -1, -1, -1, 42, 43, + 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, + 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, + 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, + 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, 232, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, + 264, 265, 266, 267, -1, 269, 270, 271, 272, -1, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, + 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, -1, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, -1, 59, - 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, + 60, 61, 62, 63, 64, -1, 66, 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, @@ -15795,85 +15855,134 @@ static const yytype_int16 yycheck[] = 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, 188, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, 203, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, 216, 217, 218, 219, - 220, 221, 222, 223, 224, 225, -1, -1, 228, 229, - 230, 231, -1, 233, 234, 235, 236, 237, 238, 239, + 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, -1, -1, 229, + 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, 261, 262, 263, 264, 265, 266, -1, 268, 269, - 270, 271, -1, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, + 260, 261, 262, 263, 264, 265, 266, 267, -1, 269, + 270, 271, 272, -1, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, 312, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 382, 383, 384, -1, 386, 387, 388, 389, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, 407, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, -1, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, 452, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, + 400, 401, 402, 403, -1, 405, 406, -1, 408, 409, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, -1, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, 455, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 3, -1, -1, -1, -1, -1, + 480, 481, 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, - 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, - -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, - 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, - 69, 70, 71, 72, 73, 74, 75, 76, 77, -1, - -1, 80, 81, 82, 83, 84, 85, 86, -1, 88, - 89, 90, 91, 92, 93, 94, 95, 96, 97, -1, - -1, 100, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, - 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, - -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, - 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, - 159, 160, 161, -1, 163, 164, 165, -1, -1, -1, - 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, -1, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, -1, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, -1, -1, 218, - -1, 220, 221, 222, 223, 224, 225, -1, -1, 228, - -1, 230, -1, -1, 233, 234, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, -1, 262, 263, 264, 265, 266, -1, 268, - 269, -1, 271, -1, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, -1, 292, -1, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, -1, 313, 314, 315, 316, 317, 318, - 319, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, -1, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, -1, 381, 382, 383, 384, -1, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, -1, 406, -1, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, -1, 424, 425, 426, 427, 428, - 429, 430, 431, -1, 433, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, -1, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, - 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 3, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, + 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, + -1, -1, -1, -1, -1, -1, 42, 43, 44, -1, + 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, + 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, + -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, + 76, 77, -1, 79, -1, 81, 82, 83, 84, 85, + 86, 87, 88, 89, 90, -1, 92, 93, 94, 95, + 96, 97, -1, 99, 100, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, + 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, + 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, + 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, + 166, -1, 168, 169, 170, 171, 172, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, 204, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, -1, -1, 229, 230, 231, 232, -1, 234, 235, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, + 266, 267, -1, 269, 270, 271, 272, -1, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, 293, 294, 295, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, 354, 355, + 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, 409, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + -1, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, 455, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, + 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, + 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, + 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, + 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, + 72, 73, 74, 75, 76, 77, -1, -1, 80, 81, + 82, 83, 84, 85, 86, -1, 88, 89, 90, 91, + 92, 93, 94, 95, 96, 97, -1, -1, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, + -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, + 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, + -1, 163, 164, 165, -1, -1, -1, 169, 170, 171, + -1, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, -1, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, -1, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, -1, -1, 219, -1, 221, + 222, 223, 224, 225, 226, -1, -1, 229, -1, 231, + -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + -1, 263, 264, 265, 266, 267, -1, 269, 270, -1, + 272, -1, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + 292, -1, 294, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, -1, 315, 316, 317, 318, 319, 320, 321, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 402, 403, -1, 405, 406, -1, 408, -1, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, -1, 427, 428, 429, 430, 431, + 432, 433, 434, -1, 436, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, -1, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, @@ -15891,185 +16000,41 @@ static const yytype_int16 yycheck[] = 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, 164, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - -1, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, -1, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, -1, -1, - 218, -1, 220, 221, 222, 223, 224, 225, -1, -1, - 228, -1, 230, 231, -1, 233, 234, 235, 236, 237, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, -1, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, -1, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, -1, + -1, 219, -1, 221, 222, 223, 224, 225, 226, -1, + -1, 229, -1, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, -1, 262, 263, 264, 265, 266, -1, - 268, 269, -1, 271, -1, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, -1, 292, -1, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, -1, -1, -1, - -1, 309, 310, 311, -1, 313, 314, 315, 316, 317, - 318, 319, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 258, 259, 260, 261, -1, 263, 264, 265, 266, 267, + -1, 269, 270, -1, 272, -1, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, -1, 294, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, -1, 315, 316, 317, + 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, -1, 353, 354, 355, 356, 357, + 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - -1, -1, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, -1, 381, 382, 383, 384, -1, 386, 387, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, -1, 406, -1, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, -1, 424, 425, 426, 427, - 428, 429, 430, 431, -1, 433, 434, 435, 436, -1, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, -1, 453, 454, 455, 456, 457, - -1, -1, 460, -1, -1, 463, 464, 465, 466, 467, - 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 3, -1, 5, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, - 27, 28, 29, -1, 31, 32, 33, 34, 35, -1, - -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, - 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, - 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, - 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, - 77, -1, -1, -1, 81, 82, 83, 84, 85, 86, - -1, 88, 89, 90, -1, 92, 93, 94, 95, 96, - 97, -1, -1, 100, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, - 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, - 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, - -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, - -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, -1, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, -1, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, -1, - -1, 218, -1, 220, 221, 222, 223, 224, 225, -1, - -1, 228, -1, 230, -1, -1, 233, 234, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, -1, 262, 263, 264, 265, 266, - -1, 268, 269, -1, 271, -1, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, -1, 292, -1, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, -1, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, -1, 353, 354, 355, 356, - 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, -1, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - -1, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, -1, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, -1, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, - 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 3, -1, 5, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, - 26, 27, 28, 29, 30, 31, 32, 33, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, - 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, - 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, - -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, - 76, 77, -1, -1, -1, 81, 82, 83, 84, 85, - 86, -1, 88, 89, 90, -1, 92, 93, 94, 95, - 96, 97, -1, -1, 100, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, - 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, - 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, - 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, - -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, -1, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, -1, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, - -1, -1, 218, -1, 220, 221, 222, 223, 224, 225, - -1, -1, 228, -1, 230, -1, -1, 233, 234, 235, - 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, -1, 262, 263, 264, 265, - 266, -1, 268, 269, -1, 271, -1, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, -1, 284, 285, - 286, -1, -1, 289, 290, -1, 292, -1, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, -1, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, -1, 353, 354, 355, - 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, -1, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, -1, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, -1, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, -1, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, -1, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, - 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 3, -1, - 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, - -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, - -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, - -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, - 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, -1, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, -1, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, -1, -1, 218, -1, 220, 221, 222, 223, 224, - 225, -1, -1, 228, -1, 230, -1, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, -1, 262, 263, 264, - 265, 266, -1, 268, 269, -1, 271, -1, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, 283, 284, - 285, 286, -1, -1, 289, 290, -1, 292, -1, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, -1, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, -1, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, -1, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, -1, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, -1, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, -1, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 3, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, -1, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, -1, 427, + 428, 429, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, -1, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, + 34, 35, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, @@ -16083,185 +16048,41 @@ static const yytype_int16 yycheck[] = 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, -1, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, -1, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, -1, -1, 218, -1, 220, 221, 222, 223, - 224, 225, -1, -1, 228, -1, 230, -1, -1, 233, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, -1, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + -1, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, -1, -1, 219, -1, 221, 222, 223, + 224, 225, 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, -1, 262, 263, - 264, 265, 266, -1, 268, 269, -1, 271, -1, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, 283, - 284, 285, 286, -1, -1, 289, 290, -1, 292, -1, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, -1, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, -1, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, -1, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, -1, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, -1, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, -1, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, -1, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, + 254, 255, 256, 257, 258, 259, 260, 261, -1, 263, + 264, 265, 266, 267, -1, 269, 270, -1, 272, -1, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, -1, + 294, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + -1, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, -1, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, -1, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, -1, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, -1, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, - 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, - 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, - 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, -1, -1, 81, 82, - 83, 84, 85, 86, -1, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, -1, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, -1, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - -1, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, -1, -1, 218, -1, 220, 221, 222, - 223, 224, 225, -1, -1, 228, -1, 230, -1, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, -1, 262, - 263, 264, 265, 266, -1, 268, 269, -1, 271, -1, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, -1, 292, - -1, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, -1, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, -1, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, -1, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, -1, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - -1, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, -1, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 3, 4, -1, -1, -1, -1, 9, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, - 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, - 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, - 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, - -1, 73, 74, 75, 76, 77, -1, -1, -1, 81, - 82, 83, 84, 85, 86, -1, 88, 89, 90, -1, - 92, 93, 94, 95, 96, 97, -1, -1, 100, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, - -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, - 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, - 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, - -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, - -1, 173, -1, 175, -1, 177, 178, 179, -1, 181, - 182, 183, 184, 185, 186, 187, -1, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, -1, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, -1, -1, 218, -1, 220, 221, - 222, 223, 224, 225, -1, -1, 228, -1, 230, -1, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, -1, - 262, 263, 264, 265, 266, -1, 268, 269, -1, 271, - -1, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, -1, - 292, -1, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - -1, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - -1, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, -1, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, -1, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, -1, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, -1, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - -1, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, - 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 3, -1, 5, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, - 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, - 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, 77, -1, -1, -1, - 81, 82, 83, 84, 85, 86, -1, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, -1, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, - 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, - 171, -1, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, -1, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, -1, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, -1, -1, 218, -1, 220, - 221, 222, 223, 224, 225, -1, -1, 228, -1, 230, - -1, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - -1, 262, 263, 264, 265, 266, -1, 268, 269, -1, - 271, -1, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - -1, 292, -1, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, -1, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, -1, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, -1, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, 405, 406, -1, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, -1, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, -1, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, - 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 3, -1, 5, -1, -1, -1, -1, + 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, + 30, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, @@ -16276,85 +16097,134 @@ static const yytype_int16 yycheck[] = 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, -1, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, -1, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, -1, -1, 218, -1, - 220, 221, 222, 223, 224, 225, -1, -1, 228, -1, - 230, -1, -1, 233, 234, 235, 236, 237, 238, 239, + 180, -1, 182, 183, 184, 185, 186, 187, 188, -1, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, -1, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, -1, -1, 219, + -1, 221, 222, 223, 224, 225, 226, -1, -1, 229, + -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, -1, 262, 263, 264, 265, 266, -1, 268, 269, - -1, 271, -1, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, -1, 292, -1, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, -1, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, -1, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - -1, 381, 382, 383, 384, -1, 386, 387, 388, 389, + 260, 261, -1, 263, 264, 265, 266, 267, -1, 269, + 270, -1, 272, -1, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, -1, 294, -1, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, -1, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, -1, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, -1, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, -1, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, + 400, 401, 402, 403, -1, 405, 406, -1, 408, -1, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, -1, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, -1, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 3, -1, 5, -1, -1, -1, + 480, 481, 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, - 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, - -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, - 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, - 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, - -1, -1, 81, 82, 83, 84, 85, 86, -1, 88, - 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, - -1, 100, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, - 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, - -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, - 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, - 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, - 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, -1, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, -1, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, -1, -1, 218, - -1, 220, 221, 222, 223, 224, 225, -1, -1, 228, - -1, 230, -1, -1, 233, 234, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, -1, 262, 263, 264, 265, 266, -1, 268, - 269, -1, 271, -1, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, -1, 292, -1, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, -1, 313, 314, 315, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, -1, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, -1, 381, 382, 383, 384, -1, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, -1, 406, -1, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, -1, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, -1, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, - 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 3, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, + 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, + 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, + 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, + -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, + 76, 77, -1, -1, -1, 81, 82, 83, 84, 85, + 86, -1, 88, 89, 90, -1, 92, 93, 94, 95, + 96, 97, -1, -1, 100, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, + 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, + 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, + 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, + -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, -1, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, -1, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, -1, -1, 219, -1, 221, 222, 223, 224, 225, + 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, -1, 263, 264, 265, + 266, 267, -1, 269, 270, -1, 272, -1, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, 284, 285, + 286, 287, -1, -1, 290, 291, 292, -1, 294, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, -1, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, -1, 355, + 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, -1, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, -1, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + -1, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, -1, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, + 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, + 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, + 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, + 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, + -1, 73, 74, 75, 76, 77, -1, -1, -1, 81, + 82, 83, 84, 85, 86, -1, 88, 89, 90, -1, + 92, 93, 94, 95, 96, 97, -1, -1, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, + -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, + 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, + -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, + -1, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, -1, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, -1, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, -1, -1, 219, -1, 221, + 222, 223, 224, 225, 226, -1, -1, 229, -1, 231, + -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + -1, 263, 264, 265, 266, 267, -1, 269, 270, -1, + 272, -1, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, 284, 285, 286, 287, -1, -1, 290, 291, + 292, -1, 294, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, -1, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 402, 403, -1, 405, 406, -1, 408, -1, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, -1, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, -1, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, -1, -1, -1, -1, @@ -16372,182 +16242,38 @@ static const yytype_int16 yycheck[] = 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - -1, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, -1, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, -1, -1, - 218, -1, 220, 221, 222, 223, 224, 225, -1, -1, - 228, -1, 230, -1, -1, 233, 234, 235, 236, 237, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, -1, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, -1, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, -1, + -1, 219, -1, 221, 222, 223, 224, 225, 226, -1, + -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, -1, 262, 263, 264, 265, 266, -1, - 268, 269, -1, 271, -1, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, -1, 292, -1, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, -1, -1, -1, - -1, 309, 310, 311, -1, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 258, 259, 260, 261, -1, 263, 264, 265, 266, 267, + -1, 269, 270, -1, 272, -1, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, -1, 294, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, -1, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, -1, 353, 354, 355, 356, 357, + 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - -1, -1, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, -1, 381, 382, 383, 384, -1, 386, 387, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, -1, 406, -1, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, -1, 424, 425, 426, 427, - 428, 429, 430, 431, -1, -1, 434, 435, 436, -1, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, -1, 453, 454, 455, 456, 457, - -1, -1, 460, -1, -1, 463, 464, 465, 466, 467, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, -1, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, -1, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, -1, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 3, -1, 5, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, - 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, - 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, - 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, - 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, - 77, -1, -1, -1, 81, 82, 83, 84, 85, 86, - -1, 88, 89, 90, -1, 92, 93, 94, 95, 96, - 97, -1, -1, 100, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, - 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, - 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, - -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, - -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, -1, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, -1, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, -1, - -1, 218, -1, 220, 221, 222, 223, 224, 225, -1, - -1, 228, -1, 230, -1, -1, 233, 234, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, -1, 262, 263, 264, 265, 266, - -1, 268, 269, -1, 271, -1, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, -1, 292, -1, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, -1, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, -1, 353, 354, 355, 356, - 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, -1, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - -1, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, -1, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, -1, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, - 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 3, -1, 5, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, - 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, - 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, - 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, - -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, - 76, 77, -1, -1, -1, 81, 82, 83, 84, 85, - 86, -1, 88, 89, 90, -1, 92, 93, 94, 95, - 96, 97, -1, -1, 100, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, - 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, - 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, - 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, - -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, -1, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, -1, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, - -1, -1, 218, -1, 220, 221, 222, 223, 224, 225, - -1, -1, 228, -1, 230, -1, -1, 233, 234, 235, - 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, -1, 262, 263, 264, 265, - 266, -1, 268, 269, -1, 271, -1, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, -1, 284, 285, - 286, -1, -1, 289, 290, -1, 292, -1, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, -1, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, -1, 353, 354, 355, - 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, -1, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, -1, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, -1, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, -1, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, -1, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, - 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 3, -1, - 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, - -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, - -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, - -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, - 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, -1, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, -1, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, -1, -1, 218, -1, 220, 221, 222, 223, 224, - 225, -1, -1, 228, -1, 230, -1, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, -1, 262, 263, 264, - 265, 266, -1, 268, 269, -1, 271, -1, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, -1, 292, -1, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, -1, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, -1, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, -1, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, -1, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, -1, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, -1, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 3, - -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, + 4, -1, -1, -1, -1, 9, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, @@ -16564,86 +16290,135 @@ static const yytype_int16 yycheck[] = 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, -1, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, -1, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, -1, -1, 218, -1, 220, 221, 222, 223, - 224, 225, -1, -1, 228, -1, 230, -1, -1, 233, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, -1, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + -1, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, -1, -1, 219, -1, 221, 222, 223, + 224, 225, 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, -1, 262, 263, - 264, 265, 266, -1, 268, 269, -1, 271, -1, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, -1, - 284, 285, 286, -1, -1, 289, 290, -1, 292, -1, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, -1, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, -1, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, -1, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, -1, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, -1, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, -1, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, -1, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, + 254, 255, 256, 257, 258, 259, 260, 261, -1, 263, + 264, 265, 266, 267, -1, 269, 270, -1, 272, -1, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, -1, + 294, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + -1, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, -1, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, -1, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, -1, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, -1, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, - 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, - 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, - 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, -1, -1, 81, 82, - 83, 84, 85, 86, -1, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, -1, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, -1, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - -1, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, -1, -1, 218, -1, 220, 221, 222, - 223, 224, 225, -1, -1, 228, -1, 230, -1, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, -1, 262, - 263, 264, 265, 266, -1, 268, 269, -1, 271, -1, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, -1, 292, - -1, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, -1, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, -1, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, -1, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, -1, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - -1, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, -1, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 3, -1, 5, -1, -1, -1, -1, -1, -1, + 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, + 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, + 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, 77, -1, -1, + -1, 81, 82, 83, 84, 85, 86, -1, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, -1, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, + 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, -1, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, -1, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, -1, -1, 219, + -1, 221, 222, 223, 224, 225, 226, -1, -1, 229, + -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, -1, 263, 264, 265, 266, 267, -1, 269, + 270, -1, 272, -1, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, -1, 294, -1, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, -1, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, -1, 405, 406, 407, 408, -1, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, -1, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, -1, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 3, -1, 5, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, + 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, + 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, + 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, + -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, + 76, 77, -1, -1, -1, 81, 82, 83, 84, 85, + 86, -1, 88, 89, 90, -1, 92, 93, 94, 95, + 96, 97, -1, -1, 100, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, + 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, + 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, + 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, + -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, -1, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, -1, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, -1, -1, 219, -1, 221, 222, 223, 224, 225, + 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, -1, 263, 264, 265, + 266, 267, -1, 269, 270, -1, 272, -1, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, -1, 294, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, -1, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, -1, 355, + 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, -1, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, -1, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + -1, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, -1, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, + 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, @@ -16660,90 +16435,139 @@ static const yytype_int16 yycheck[] = 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, - -1, 173, -1, 175, -1, 177, 178, 179, -1, 181, - 182, 183, 184, 185, 186, 187, -1, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, -1, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, -1, -1, 218, -1, 220, 221, - 222, 223, 224, 225, -1, -1, 228, -1, 230, -1, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, + -1, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, -1, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, -1, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, -1, -1, 219, -1, 221, + 222, 223, 224, 225, 226, -1, -1, 229, -1, 231, + -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, -1, - 262, 263, 264, 265, 266, -1, 268, 269, -1, 271, - -1, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, -1, - 292, -1, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - -1, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - -1, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, -1, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, -1, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + -1, 263, 264, 265, 266, 267, -1, 269, 270, -1, + 272, -1, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + 292, -1, 294, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, -1, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, -1, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, -1, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - -1, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, + 402, 403, -1, 405, 406, -1, 408, -1, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, -1, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, -1, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 3, -1, 5, -1, -1, -1, -1, -1, + 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, - 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, - 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, 77, -1, -1, -1, - 81, 82, 83, 84, 85, 86, -1, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, -1, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, - 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, - 171, -1, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, -1, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, -1, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, -1, -1, 218, -1, 220, - 221, 222, 223, 224, 225, -1, -1, 228, -1, 230, - -1, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - -1, 262, 263, 264, 265, 266, -1, 268, 269, -1, - 271, -1, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - -1, 292, -1, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, -1, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, -1, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, -1, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, -1, 406, -1, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, -1, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, -1, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, - 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 3, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, + 28, 29, 30, 31, 32, 33, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, + 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, + -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, + 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, + -1, -1, -1, 81, 82, 83, 84, 85, 86, -1, + 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, + -1, -1, 100, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, + 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, + -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, + 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, + 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, + -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, -1, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, -1, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, -1, + -1, 219, -1, 221, 222, 223, 224, 225, 226, -1, + -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 258, 259, 260, 261, -1, 263, 264, 265, 266, 267, + -1, 269, 270, -1, 272, -1, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, -1, 294, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, -1, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, + 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, -1, 383, 384, 385, 386, -1, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, -1, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, -1, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, -1, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, + -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, + 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, + 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, + 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, -1, -1, 81, 82, 83, + 84, 85, 86, -1, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, -1, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, -1, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + -1, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, -1, -1, 219, -1, 221, 222, 223, + 224, 225, 226, -1, -1, 229, -1, 231, -1, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, -1, 263, + 264, 265, 266, 267, -1, 269, 270, -1, 272, -1, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, -1, + 294, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + -1, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, -1, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, -1, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, -1, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, -1, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, - 40, -1, -1, 43, 44, -1, 46, 47, 48, -1, + -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, -1, @@ -16757,88 +16581,137 @@ static const yytype_int16 yycheck[] = 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, -1, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, -1, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, -1, -1, 218, -1, - 220, 221, 222, 223, 224, 225, -1, -1, 228, -1, - 230, -1, -1, 233, 234, 235, 236, 237, 238, 239, + 180, -1, 182, 183, 184, 185, 186, 187, 188, -1, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, -1, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, -1, -1, 219, + -1, 221, 222, 223, 224, 225, 226, -1, -1, 229, + -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, -1, 262, 263, 264, 265, 266, -1, 268, 269, - -1, 271, -1, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, -1, 292, -1, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, -1, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, -1, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - -1, 381, 382, 383, 384, -1, 386, 387, 388, 389, + 260, 261, -1, 263, 264, 265, 266, 267, -1, 269, + 270, -1, 272, -1, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, -1, 294, -1, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, -1, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, -1, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, -1, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, -1, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, + 400, 401, 402, 403, -1, 405, 406, -1, 408, -1, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, -1, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, -1, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 3, -1, -1, -1, -1, -1, + 480, 481, 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, - 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, - -1, 40, -1, -1, 43, 44, -1, 46, 47, 48, - -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, - 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, - 69, 70, 71, -1, 73, 74, 75, 76, 77, -1, - -1, -1, 81, 82, 83, 84, 85, 86, -1, 88, - 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, - -1, 100, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, - 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, - -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, 147, 148, - 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, - 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, - 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, -1, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, -1, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, -1, -1, 218, - -1, 220, 221, 222, 223, 224, 225, -1, -1, 228, - -1, 230, -1, -1, 233, 234, 235, 236, 237, 238, - 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, -1, 262, 263, 264, 265, 266, -1, 268, - 269, -1, 271, -1, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, -1, 292, -1, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, -1, 313, 314, 315, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, - 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, -1, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, -1, 381, 382, 383, 384, -1, 386, 387, 388, - 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, -1, 406, -1, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, -1, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, -1, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, - 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 3, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, + 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, + 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, + 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, + -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, + 76, 77, -1, -1, -1, 81, 82, 83, 84, 85, + 86, -1, 88, 89, 90, -1, 92, 93, 94, 95, + 96, 97, -1, -1, 100, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, + 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, + 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, + 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, + -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, -1, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, -1, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, -1, -1, 219, -1, 221, 222, 223, 224, 225, + 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, + 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 258, 259, 260, 261, -1, 263, 264, 265, + 266, 267, -1, 269, 270, -1, 272, -1, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, -1, 294, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, -1, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, -1, 355, + 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, -1, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, -1, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + -1, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, -1, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, + 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, + 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, + 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, + 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, + -1, 73, 74, 75, 76, 77, -1, -1, -1, 81, + 82, 83, 84, 85, 86, -1, 88, 89, 90, -1, + 92, 93, 94, 95, 96, 97, -1, -1, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, + -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, + 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, + -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, + -1, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, -1, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, -1, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, -1, -1, 219, -1, 221, + 222, 223, 224, 225, 226, -1, -1, 229, -1, 231, + -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + -1, 263, 264, 265, 266, 267, -1, 269, 270, -1, + 272, -1, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + 292, -1, 294, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, -1, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 402, 403, -1, 405, 406, -1, 408, -1, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, -1, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, -1, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, - 28, 29, 30, 31, 32, 33, -1, -1, -1, -1, + 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, @@ -16853,89 +16726,138 @@ static const yytype_int16 yycheck[] = 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - -1, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, -1, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, -1, -1, - 218, -1, 220, 221, 222, 223, 224, 225, -1, -1, - 228, -1, 230, -1, -1, 233, 234, 235, 236, 237, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, -1, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, -1, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, -1, + -1, 219, -1, 221, 222, 223, 224, 225, 226, -1, + -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, -1, 262, 263, 264, 265, 266, -1, - 268, 269, -1, 271, -1, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, -1, 292, -1, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, -1, -1, -1, - -1, 309, 310, 311, -1, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 258, 259, 260, 261, -1, 263, 264, 265, 266, 267, + -1, 269, 270, -1, 272, -1, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, -1, 294, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, -1, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, -1, 353, 354, 355, 356, 357, + 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - -1, -1, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, -1, 381, 382, 383, 384, -1, 386, 387, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, -1, 406, -1, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, -1, 424, 425, 426, 427, - 428, 429, 430, 431, -1, -1, 434, 435, 436, -1, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, -1, 453, 454, 455, 456, 457, - -1, -1, 460, -1, -1, 463, 464, 465, 466, 467, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, -1, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, -1, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, -1, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 3, -1, 5, -1, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, + -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, + 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, + 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, + 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, 77, -1, -1, -1, 81, 82, 83, + 84, 85, 86, -1, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, -1, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, -1, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + -1, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, -1, -1, 219, -1, 221, 222, 223, + 224, 225, 226, -1, -1, 229, -1, 231, -1, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, -1, 263, + 264, 265, 266, 267, -1, 269, 270, -1, 272, -1, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, -1, + 294, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + -1, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, -1, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, -1, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, -1, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, -1, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, - 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, - 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, - 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, - 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, - 77, -1, -1, -1, 81, 82, 83, 84, 85, 86, - -1, 88, 89, 90, -1, 92, 93, 94, 95, 96, - 97, -1, -1, 100, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, - 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, - 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, - -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, - -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, - 177, 178, 179, -1, 181, 182, 183, 184, 185, 186, - 187, -1, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, -1, 204, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, -1, - -1, 218, -1, 220, 221, 222, 223, 224, 225, -1, - -1, 228, -1, 230, -1, -1, 233, 234, 235, 236, - 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, -1, 262, 263, 264, 265, 266, - -1, 268, 269, -1, 271, -1, 273, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, 290, -1, 292, -1, 294, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, 311, -1, 313, 314, 315, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, -1, 353, 354, 355, 356, - 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, -1, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, -1, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, -1, 403, 404, -1, 406, - -1, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, 420, 421, 422, -1, 424, 425, 426, - 427, 428, 429, 430, 431, -1, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, 448, 449, -1, 451, -1, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, - 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 3, -1, 5, + -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, + 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, + 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, 77, -1, -1, + -1, 81, 82, 83, 84, 85, 86, -1, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, -1, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, + 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, -1, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, -1, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, -1, -1, 219, + -1, 221, 222, 223, 224, 225, 226, -1, -1, 229, + -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, -1, 263, 264, 265, 266, 267, -1, 269, + 270, -1, 272, -1, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, -1, 294, -1, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, -1, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, -1, 405, 406, -1, 408, -1, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, -1, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, -1, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, + -1, -1, -1, -1, 40, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, @@ -16949,85 +16871,134 @@ static const yytype_int16 yycheck[] = 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, -1, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, -1, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, - -1, -1, 218, -1, 220, 221, 222, 223, 224, 225, - -1, -1, 228, -1, 230, -1, -1, 233, 234, 235, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, -1, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, -1, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, -1, -1, 219, -1, 221, 222, 223, 224, 225, + 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, -1, 262, 263, 264, 265, - 266, -1, 268, 269, -1, 271, -1, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, -1, 284, 285, - 286, -1, -1, 289, 290, -1, 292, -1, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, -1, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, -1, 353, 354, 355, + 256, 257, 258, 259, 260, 261, -1, 263, 264, 265, + 266, 267, -1, 269, 270, -1, 272, -1, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, -1, 294, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, -1, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, -1, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, -1, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, -1, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, -1, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, -1, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, -1, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, -1, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + -1, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, -1, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 3, -1, - 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, - -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, - -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, - -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, - 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, -1, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, -1, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, -1, -1, 218, -1, 220, 221, 222, 223, 224, - 225, -1, -1, 228, -1, 230, -1, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, -1, 262, 263, 264, - 265, 266, -1, 268, 269, -1, 271, -1, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, -1, 292, -1, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, -1, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, -1, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, -1, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, -1, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, -1, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, -1, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 3, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, + 32, 33, -1, -1, -1, -1, -1, -1, 40, -1, + -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, + 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, + 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, + -1, 73, 74, 75, 76, 77, -1, -1, -1, 81, + 82, 83, 84, 85, 86, -1, 88, 89, 90, -1, + 92, 93, 94, 95, 96, 97, -1, -1, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, + -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, + 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, + 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, + -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, + -1, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, -1, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, -1, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, -1, -1, 219, -1, 221, + 222, 223, 224, 225, 226, -1, -1, 229, -1, 231, + -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + -1, 263, 264, 265, 266, 267, -1, 269, 270, -1, + 272, -1, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + 292, -1, 294, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, -1, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + 402, 403, -1, 405, 406, -1, 408, -1, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, -1, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, -1, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, + 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, + 28, 29, 30, 31, 32, 33, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, + 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, + -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, + 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, + -1, -1, -1, 81, 82, 83, 84, 85, 86, -1, + 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, + -1, -1, 100, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, + 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, + -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, + 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, + 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, + -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, -1, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, -1, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, -1, + -1, 219, -1, 221, 222, 223, 224, 225, 226, -1, + -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 258, 259, 260, 261, -1, 263, 264, 265, 266, 267, + -1, 269, 270, -1, 272, -1, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, -1, 294, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, -1, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, + 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, -1, 383, 384, 385, 386, -1, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, -1, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, -1, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, -1, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, @@ -17045,134 +17016,88 @@ static const yytype_int16 yycheck[] = 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, -1, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, -1, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, -1, -1, 218, -1, 220, 221, 222, 223, - 224, 225, -1, -1, 228, -1, 230, -1, -1, 233, + -1, 175, -1, 177, 178, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, -1, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + -1, 205, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, -1, -1, 219, -1, 221, 222, 223, + 224, 225, 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, -1, 262, 263, - 264, 265, 266, -1, 268, 269, -1, 271, -1, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, -1, - 284, 285, 286, -1, -1, 289, 290, -1, 292, -1, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, -1, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, -1, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, -1, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, -1, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, -1, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, -1, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, -1, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, + 254, 255, 256, 257, 258, 259, 260, 261, -1, 263, + 264, 265, 266, 267, -1, 269, 270, -1, 272, -1, + 274, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, 292, -1, + 294, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, 313, + -1, 315, 316, 317, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, -1, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, -1, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, + -1, 405, 406, -1, 408, -1, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, 423, + 424, 425, -1, 427, 428, 429, 430, 431, 432, 433, + 434, -1, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, 451, 452, -1, + 454, -1, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, - 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, - 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, - 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, -1, -1, 81, 82, - 83, 84, 85, 86, -1, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, -1, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, -1, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - -1, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, -1, -1, 218, -1, 220, 221, 222, - 223, 224, 225, -1, -1, 228, -1, 230, -1, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, -1, 262, - 263, 264, 265, 266, -1, 268, 269, -1, 271, -1, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, -1, 292, - -1, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, -1, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, -1, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, -1, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, -1, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - -1, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, -1, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 3, -1, -1, -1, -1, -1, -1, -1, -1, + 484, 485, 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, - 32, 33, -1, -1, -1, 37, -1, -1, 40, -1, - 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, - 52, -1, 54, 55, 56, 57, -1, 59, 60, 61, - 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, - -1, 73, 74, 75, 76, -1, -1, 79, -1, 81, - 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, - 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, - -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, - 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, 147, -1, 149, 150, 151, - 152, 153, -1, -1, 156, -1, 158, 159, 160, 161, - -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, - 172, 173, -1, 175, -1, -1, -1, 179, -1, 181, - 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, 203, -1, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, -1, -1, 228, 229, 230, 231, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 264, 265, -1, -1, 268, 269, 270, 271, - -1, -1, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, -1, 291, - 292, 293, -1, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, -1, - 312, 313, 314, -1, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, 369, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, 398, 399, -1, 401, - -1, 403, 404, -1, 406, 407, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, -1, 421, - -1, -1, 424, 425, 426, 427, 428, 429, 430, 431, - 432, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, -1, 449, -1, 451, - 452, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 3, -1, 5, - -1, -1, -1, -1, -1, -1, -1, -1, 480, 481, - 482, 483, -1, -1, -1, -1, 22, 23, 24, 25, + -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, + 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, + 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, 77, -1, -1, + -1, 81, 82, 83, 84, 85, 86, -1, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, -1, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, + 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, + 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, -1, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, -1, 205, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, -1, -1, 219, + -1, 221, 222, 223, 224, 225, 226, -1, -1, 229, + -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, -1, 263, 264, 265, 266, 267, -1, 269, + 270, -1, 272, -1, 274, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, 292, -1, 294, -1, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, 313, -1, 315, 316, 317, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, -1, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, 402, 403, -1, 405, 406, -1, 408, -1, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, 423, 424, 425, -1, 427, 428, 429, + 430, 431, 432, 433, 434, -1, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, 451, 452, -1, 454, -1, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, + 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 3, -1, 5, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, @@ -17188,182 +17113,38 @@ static const yytype_int16 yycheck[] = 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, - -1, 177, 178, 179, -1, 181, 182, 183, 184, 185, - 186, 187, -1, 189, 190, 191, 192, -1, 194, 195, - 196, 197, 198, 199, 200, -1, 202, -1, 204, 205, - 206, 207, 208, 209, 210, 211, -1, 213, -1, 215, - -1, -1, 218, -1, 220, 221, 222, 223, 224, 225, - -1, -1, 228, -1, 230, -1, -1, 233, 234, 235, + -1, 177, 178, 179, 180, -1, 182, 183, 184, 185, + 186, 187, 188, -1, 190, 191, 192, 193, -1, 195, + 196, 197, 198, 199, 200, 201, -1, 203, -1, 205, + 206, 207, 208, 209, 210, 211, 212, -1, 214, -1, + 216, -1, -1, 219, -1, 221, 222, 223, 224, 225, + 226, -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, - 256, 257, 258, 259, 260, -1, 262, 263, 264, 265, - 266, -1, 268, 269, -1, 271, -1, 273, 274, 275, - 276, 277, 278, -1, 280, 281, -1, -1, 284, 285, - 286, -1, -1, 289, 290, -1, 292, -1, 294, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, -1, - -1, -1, -1, 309, 310, 311, -1, 313, 314, 315, - 316, 317, 318, -1, 320, 321, 322, 323, 324, 325, - -1, 327, 328, 329, 330, 331, 332, 333, 334, -1, - 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, - 346, 347, 348, -1, 350, 351, -1, 353, 354, 355, + 256, 257, 258, 259, 260, 261, -1, 263, 264, 265, + 266, 267, -1, 269, 270, -1, 272, -1, 274, 275, + 276, 277, 278, 279, -1, 281, 282, -1, -1, 285, + 286, 287, -1, -1, 290, 291, 292, -1, 294, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, -1, -1, -1, -1, 311, 312, 313, -1, 315, + 316, 317, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, 335, + 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, + 346, 347, 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, -1, -1, 370, 371, 372, 373, 374, 375, - 376, 377, 378, 379, -1, 381, 382, 383, 384, -1, - 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, -1, 403, 404, -1, - 406, -1, 408, 409, 410, 411, 412, -1, 414, 415, - -1, -1, 418, 419, 420, 421, 422, -1, 424, 425, - 426, 427, 428, 429, 430, 431, -1, -1, 434, 435, - 436, -1, 438, 439, 440, 441, -1, 443, 444, 445, - 446, 447, 448, 449, -1, 451, -1, 453, 454, 455, - 456, 457, -1, -1, 460, -1, -1, 463, 464, 465, + 366, 367, 368, 369, -1, -1, 372, 373, 374, 375, + 376, 377, 378, 379, 380, 381, -1, 383, 384, 385, + 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 397, 398, 399, 400, 401, 402, 403, -1, 405, + 406, -1, 408, -1, 410, 411, 412, 413, 414, -1, + 416, 417, 418, -1, -1, 421, 422, 423, 424, 425, + -1, 427, 428, 429, 430, 431, 432, 433, 434, -1, + -1, 437, 438, 439, -1, 441, 442, 443, 444, -1, + 446, 447, 448, 449, 450, 451, 452, -1, 454, -1, + 456, 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 3, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, - -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, - -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, - -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, - 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, - 175, -1, 177, 178, 179, -1, 181, 182, 183, 184, - 185, 186, 187, -1, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, -1, 204, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, -1, -1, 218, -1, 220, 221, 222, 223, 224, - 225, -1, -1, 228, -1, 230, -1, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, -1, 262, 263, 264, - 265, 266, -1, 268, 269, -1, 271, -1, 273, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, 290, -1, 292, -1, 294, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, 311, -1, 313, 314, - 315, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, -1, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, -1, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, -1, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, 400, 401, -1, 403, 404, - -1, 406, -1, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, 420, 421, 422, -1, 424, - 425, 426, 427, 428, 429, 430, 431, -1, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, 448, 449, -1, 451, -1, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 3, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, - 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, - 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, - 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, - 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, - 74, 75, 76, 77, -1, -1, -1, 81, 82, 83, - 84, 85, 86, -1, 88, 89, 90, -1, 92, 93, - 94, 95, 96, 97, -1, -1, 100, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, - 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, - 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, - 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, - 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, - -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, - -1, 175, -1, 177, 178, 179, -1, 181, 182, 183, - 184, 185, 186, 187, -1, 189, 190, 191, 192, -1, - 194, 195, 196, 197, 198, 199, 200, -1, 202, -1, - 204, 205, 206, 207, 208, 209, 210, 211, -1, 213, - -1, 215, -1, -1, 218, -1, 220, 221, 222, 223, - 224, 225, -1, -1, 228, -1, 230, -1, -1, 233, - 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 256, 257, 258, 259, 260, -1, 262, 263, - 264, 265, 266, -1, 268, 269, -1, 271, -1, 273, - 274, 275, 276, 277, 278, -1, 280, 281, -1, -1, - 284, 285, 286, -1, -1, 289, 290, -1, 292, -1, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, -1, -1, -1, -1, 309, 310, 311, -1, 313, - 314, 315, 316, 317, 318, -1, 320, 321, 322, 323, - 324, 325, -1, 327, 328, 329, 330, 331, 332, 333, - 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, - 344, 345, 346, 347, 348, -1, 350, 351, -1, 353, - 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, -1, -1, 370, 371, 372, 373, - 374, 375, 376, 377, 378, 379, -1, 381, 382, 383, - 384, -1, 386, 387, 388, 389, 390, 391, 392, 393, - 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, - 404, -1, 406, -1, 408, 409, 410, 411, 412, -1, - 414, 415, -1, -1, 418, 419, 420, 421, 422, -1, - 424, 425, 426, 427, 428, 429, 430, 431, -1, -1, - 434, 435, 436, -1, 438, 439, 440, 441, -1, 443, - 444, 445, 446, 447, 448, 449, -1, 451, -1, 453, - 454, 455, 456, 457, -1, -1, 460, -1, -1, 463, - 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, - 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, - 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, - 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, - 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, - 73, 74, 75, 76, 77, -1, -1, -1, 81, 82, - 83, 84, 85, 86, -1, 88, 89, 90, -1, 92, - 93, 94, 95, 96, 97, -1, -1, 100, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, - 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, - 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, - 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, - 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, -1, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - -1, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, -1, -1, 218, -1, 220, 221, 222, - 223, 224, 225, -1, -1, 228, -1, 230, -1, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, -1, 262, - 263, 264, 265, 266, -1, 268, 269, -1, 271, -1, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, -1, 292, - -1, 294, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, -1, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, 328, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, -1, - 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, -1, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, -1, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - -1, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, -1, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, - 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 3, -1, -1, -1, -1, -1, -1, -1, -1, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, @@ -17380,41 +17161,185 @@ static const yytype_int16 yycheck[] = 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, - -1, 173, -1, 175, -1, 177, 178, 179, -1, 181, - 182, 183, 184, 185, 186, 187, -1, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, -1, 204, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, -1, -1, 218, -1, 220, 221, - 222, 223, 224, 225, -1, -1, 228, -1, 230, -1, - -1, 233, 234, 235, 236, 237, 238, 239, 240, 241, + -1, 173, -1, 175, -1, 177, 178, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, -1, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, -1, 205, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, -1, -1, 219, -1, 221, + 222, 223, 224, 225, 226, -1, -1, 229, -1, 231, + -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, -1, - 262, 263, 264, 265, 266, -1, 268, 269, -1, 271, - -1, 273, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, 290, -1, - 292, -1, 294, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, 311, - -1, 313, 314, 315, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - -1, 353, 354, 355, 356, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, -1, 370, 371, - 372, 373, 374, 375, 376, 377, 378, 379, -1, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + -1, 263, 264, 265, 266, 267, -1, 269, 270, -1, + 272, -1, 274, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + 292, -1, 294, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, 313, -1, 315, 316, 317, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, -1, + 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, + -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, - -1, 403, 404, -1, 406, -1, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, 420, 421, - 422, -1, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, 448, 449, -1, 451, - -1, 453, 454, 455, 456, 457, -1, -1, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, + 402, 403, -1, 405, 406, -1, 408, -1, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, 423, 424, 425, -1, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, 451, + 452, -1, 454, -1, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 3, -1, -1, -1, -1, -1, -1, -1, + 482, 483, 484, 485, 486, 3, -1, 5, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, + 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, + 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, + -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, + 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, + -1, -1, -1, 81, 82, 83, 84, 85, 86, -1, + 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, + -1, -1, 100, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, + 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, + -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, + 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, + 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, + -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, + 178, 179, 180, -1, 182, 183, 184, 185, 186, 187, + 188, -1, 190, 191, 192, 193, -1, 195, 196, 197, + 198, 199, 200, 201, -1, 203, -1, 205, 206, 207, + 208, 209, 210, 211, 212, -1, 214, -1, 216, -1, + -1, 219, -1, 221, 222, 223, 224, 225, 226, -1, + -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, + 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 258, 259, 260, 261, -1, 263, 264, 265, 266, 267, + -1, 269, 270, -1, 272, -1, 274, 275, 276, 277, + 278, 279, -1, 281, 282, -1, -1, 285, 286, 287, + -1, -1, 290, 291, 292, -1, 294, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, -1, + -1, -1, -1, 311, 312, 313, -1, 315, 316, 317, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, 335, 336, -1, + 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, + 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, -1, -1, 372, 373, 374, 375, 376, 377, + 378, 379, 380, 381, -1, 383, 384, 385, 386, -1, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, + 398, 399, 400, 401, 402, 403, -1, 405, 406, -1, + 408, -1, 410, 411, 412, 413, 414, -1, 416, 417, + 418, -1, -1, 421, 422, 423, 424, 425, -1, 427, + 428, 429, 430, 431, 432, 433, 434, -1, -1, 437, + 438, 439, -1, 441, 442, 443, 444, -1, 446, 447, + 448, 449, 450, 451, 452, -1, 454, -1, 456, 457, + 458, 459, 460, -1, -1, 463, -1, -1, 466, 467, + 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 3, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, + 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, + -1, -1, -1, 37, -1, -1, 40, -1, 42, 43, + 44, -1, 46, 47, 48, 49, 50, 51, 52, -1, + 54, 55, 56, 57, -1, 59, 60, 61, 62, 63, + 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, + 74, 75, 76, -1, -1, 79, -1, 81, 82, 83, + 84, 85, 86, 87, 88, 89, 90, -1, 92, 93, + 94, 95, 96, 97, -1, 99, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, + 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, + 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, + 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, -1, 149, 150, 151, 152, 153, + -1, -1, 156, -1, 158, 159, 160, 161, -1, 163, + -1, 165, 166, -1, 168, 169, 170, 171, 172, 173, + -1, 175, -1, -1, -1, 179, 180, -1, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + -1, 195, 196, 197, 198, 199, 200, 201, -1, 203, + 204, -1, 206, 207, 208, 209, 210, 211, 212, -1, + 214, -1, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, -1, -1, 229, 230, 231, 232, -1, + 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, + 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, + 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, + 264, 265, 266, -1, -1, 269, 270, 271, 272, -1, + -1, 275, 276, 277, 278, 279, -1, 281, 282, -1, + -1, 285, 286, 287, -1, -1, 290, 291, -1, 293, + 294, 295, -1, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, -1, -1, -1, -1, 311, 312, -1, + 314, 315, 316, -1, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, + 344, 345, 346, 347, 348, 349, 350, -1, 352, 353, + 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, -1, 371, 372, 373, + 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, + 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 397, 398, 399, 400, 401, -1, 403, + -1, 405, 406, -1, 408, 409, 410, 411, 412, 413, + 414, -1, 416, 417, 418, -1, -1, 421, 422, -1, + 424, -1, -1, 427, 428, 429, 430, 431, 432, 433, + 434, 435, -1, 437, 438, 439, -1, 441, 442, 443, + 444, -1, 446, 447, 448, 449, 450, -1, 452, -1, + 454, 455, 456, 457, 458, 459, 460, -1, -1, 463, + -1, -1, 466, 467, 468, 469, 470, 471, 3, -1, + 5, -1, -1, -1, -1, -1, -1, -1, -1, 483, + 484, 485, 486, -1, -1, -1, -1, 22, 23, 24, + 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, + -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, + -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, + -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, + 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, + 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, + 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, + -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, + 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, + -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, + -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, + 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, -1, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, -1, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, -1, -1, 219, -1, 221, 222, 223, 224, + 225, 226, -1, -1, 229, -1, 231, -1, -1, 234, + 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, + 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 258, 259, 260, 261, -1, 263, 264, + 265, 266, 267, -1, 269, 270, -1, 272, -1, 274, + 275, 276, 277, 278, 279, -1, 281, 282, -1, -1, + 285, 286, 287, -1, -1, 290, 291, 292, -1, 294, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, -1, -1, -1, -1, 311, 312, 313, -1, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, -1, + 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, + 375, 376, 377, 378, 379, 380, 381, -1, 383, 384, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, -1, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, -1, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + -1, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, -1, -1, -1, 40, + 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, @@ -17428,86 +17353,135 @@ static const yytype_int16 yycheck[] = 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, - 171, -1, 173, -1, 175, -1, 177, 178, 179, -1, - 181, 182, 183, 184, 185, 186, 187, -1, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, -1, 204, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, -1, -1, 218, -1, 220, - 221, 222, 223, 224, 225, -1, -1, 228, -1, 230, - -1, -1, 233, 234, 235, 236, 237, 238, 239, 240, + 171, -1, 173, -1, 175, -1, 177, 178, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, -1, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, -1, 205, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, -1, -1, 219, -1, + 221, 222, 223, 224, 225, 226, -1, -1, 229, -1, + 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - -1, 262, 263, 264, 265, 266, -1, 268, 269, -1, - 271, -1, 273, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, 290, - -1, 292, -1, 294, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - 311, -1, 313, 314, 315, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, -1, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, -1, 370, - 371, 372, 373, 374, -1, 376, 377, 378, 379, -1, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, + 261, -1, 263, 264, 265, 266, 267, -1, 269, 270, + -1, 272, -1, 274, 275, 276, 277, 278, 279, -1, + 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, + 291, 292, -1, 294, -1, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, 313, -1, 315, 316, 317, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, -1, 403, 404, -1, 406, -1, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, 420, - 421, 422, -1, 424, 425, 426, 427, 428, 429, 430, - 431, -1, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, 448, 449, -1, - 451, -1, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 469, 470, + 401, 402, 403, -1, 405, 406, -1, 408, -1, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, + 421, 422, 423, 424, 425, -1, 427, 428, 429, 430, + 431, 432, 433, 434, -1, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + 451, 452, -1, 454, -1, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 3, -1, -1, -1, -1, -1, -1, + 481, 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, - -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, - 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, - 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, - 70, 71, -1, 73, 74, 75, 76, 77, -1, -1, - -1, 81, 82, 83, 84, 85, 86, -1, 88, 89, - 90, -1, 92, 93, 94, 95, 96, 97, -1, -1, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, - 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, - 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, 147, 148, 149, - 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, - 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, - 170, 171, -1, 173, -1, 175, -1, 177, 178, 179, - -1, 181, 182, 183, 184, 185, 186, 187, -1, 189, - 190, 191, 192, -1, 194, 195, 196, 197, 198, 199, - 200, -1, 202, -1, 204, 205, 206, 207, 208, 209, - 210, 211, -1, 213, -1, 215, -1, -1, 218, -1, - 220, 221, 222, 223, 224, 225, -1, -1, 228, -1, - 230, -1, -1, 233, 234, 235, 236, 237, 238, 239, - 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, - 260, -1, 262, 263, 264, 265, 266, -1, 268, 269, - -1, 271, -1, 273, 274, 275, 276, 277, 278, -1, - 280, 281, -1, -1, 284, 285, 286, -1, -1, 289, - 290, -1, 292, -1, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, -1, -1, -1, -1, 309, - 310, 311, -1, 313, 314, 315, 316, 317, 318, -1, - 320, 321, 322, 323, 324, 325, -1, 327, 328, 329, - 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, - 340, 341, 342, 343, 344, 345, 346, 347, 348, -1, - 350, 351, -1, 353, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, -1, -1, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - -1, 381, 382, 383, 384, -1, 386, 387, 388, 389, - 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, - 400, 401, -1, 403, 404, -1, 406, -1, 408, 409, - 410, 411, 412, -1, 414, 415, -1, -1, 418, 419, - 420, 421, 422, -1, 424, 425, 426, 427, 428, 429, - 430, 431, -1, -1, 434, 435, 436, -1, 438, 439, - 440, 441, -1, 443, 444, 445, 446, 447, 448, 449, - -1, 451, -1, 453, 454, 455, 456, 457, -1, -1, - 460, -1, -1, 463, 464, 465, 466, 467, 468, 469, - 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 3, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, + 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, + 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, + 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, + 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, + 77, -1, -1, -1, 81, 82, 83, 84, 85, 86, + -1, 88, 89, 90, -1, 92, 93, 94, 95, 96, + 97, -1, -1, 100, 101, 102, 103, 104, 105, 106, + 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, + 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, + 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, + 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, + 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, + -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, + -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, -1, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, -1, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + -1, -1, 219, -1, 221, 222, 223, 224, 225, 226, + -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, + 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, + 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, + 257, 258, 259, 260, 261, -1, 263, 264, 265, 266, + 267, -1, 269, 270, -1, 272, -1, 274, 275, 276, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, -1, 294, -1, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, -1, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, -1, 355, 356, + 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, + 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, -1, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, -1, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, -1, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, -1, 454, -1, 456, + 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, + 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, + 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, + 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, + 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, + 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, + 73, 74, 75, 76, 77, -1, -1, -1, 81, 82, + 83, 84, 85, 86, -1, 88, 89, 90, -1, 92, + 93, 94, 95, 96, 97, -1, -1, 100, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, + 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, + 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, + 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, + 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, + 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, + 183, 184, 185, 186, 187, 188, -1, 190, 191, 192, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, -1, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, -1, -1, 219, -1, 221, 222, + 223, 224, 225, 226, -1, -1, 229, -1, 231, -1, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, + 253, 254, 255, 256, 257, 258, 259, 260, 261, -1, + 263, 264, 265, 266, 267, -1, 269, 270, -1, 272, + -1, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, + -1, 294, -1, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, -1, -1, -1, -1, 311, 312, + 313, -1, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, + 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, -1, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, -1, 408, -1, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, -1, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, -1, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, -1, 456, 457, 458, 459, 460, -1, -1, + 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, + 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, @@ -17525,226 +17499,232 @@ static const yytype_int16 yycheck[] = 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, 178, - 179, -1, 181, 182, 183, 184, 185, 186, 187, -1, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, -1, 204, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, -1, -1, 218, - -1, 220, 221, 222, 223, 224, 225, -1, -1, 228, - -1, 230, -1, -1, 233, 234, 235, 236, 237, 238, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + -1, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, -1, 205, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, -1, -1, + 219, -1, 221, 222, 223, 224, 225, 226, -1, -1, + 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, -1, 262, 263, 264, 265, 266, -1, 268, - 269, -1, 271, -1, 273, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, 290, -1, 292, -1, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, 311, -1, 313, 314, 315, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, + 259, 260, 261, -1, 263, 264, 265, 266, 267, -1, + 269, 270, -1, 272, -1, 274, 275, 276, 277, 278, + 279, -1, 281, 282, -1, -1, 285, 286, 287, -1, + -1, 290, 291, 292, -1, 294, -1, 296, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, 313, -1, 315, 316, 317, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, -1, 353, 354, 355, 356, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - -1, 370, 371, 372, 373, 374, 375, 376, 377, 378, - 379, -1, 381, 382, 383, 384, -1, 386, 387, 388, + 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, -1, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, 400, 401, -1, 403, 404, -1, 406, -1, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, 420, 421, 422, -1, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, 448, - 449, -1, 451, -1, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, + 399, 400, 401, 402, 403, -1, 405, 406, -1, 408, + -1, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, 423, 424, 425, -1, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, 451, 452, -1, 454, -1, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 3, -1, -1, -1, -1, + 479, 480, 481, 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, - 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, - 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, - -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, - 68, 69, 70, 71, -1, 73, 74, 75, 76, 77, - -1, -1, -1, 81, 82, 83, 84, 85, 86, -1, - 88, 89, 90, -1, 92, 93, 94, 95, 96, 97, - -1, -1, 100, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, - 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, - -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, - 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, - 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, - -1, 169, 170, 171, -1, 173, -1, 175, -1, 177, - 178, 179, -1, 181, 182, 183, 184, 185, 186, 187, - -1, 189, 190, 191, 192, -1, 194, 195, 196, 197, - 198, 199, 200, -1, 202, -1, 204, 205, 206, 207, - 208, 209, 210, 211, -1, 213, -1, 215, -1, -1, - 218, -1, 220, 221, 222, 223, 224, 225, -1, -1, - 228, -1, 230, -1, -1, 233, 234, 235, 236, 237, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, - 258, 259, 260, -1, 262, 263, 264, 265, 266, -1, - 268, 269, -1, 271, -1, 273, 274, 275, 276, 277, - 278, -1, 280, 281, -1, -1, 284, 285, 286, -1, - -1, 289, 290, -1, 292, -1, 294, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, -1, -1, -1, - -1, 309, 310, 311, -1, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, 323, 324, 325, -1, 327, - 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, - 348, -1, 350, 351, -1, 353, 354, 355, 356, 357, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - -1, -1, 370, 371, 372, 373, 374, 375, 376, 377, - 378, 379, -1, 381, 382, 383, 384, -1, 386, 387, - 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, -1, 403, 404, -1, 406, -1, - 408, 409, 410, 411, 412, -1, 414, 415, -1, -1, - 418, 419, 420, 421, 422, -1, 424, 425, 426, 427, - 428, 429, 430, 431, -1, -1, 434, 435, 436, -1, - 438, 439, 440, 441, -1, 443, 444, 445, 446, 447, - 448, 449, -1, 451, -1, 453, 454, 455, 456, 457, - -1, -1, 460, -1, -1, 463, 464, 465, 466, 467, - 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 3, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, + 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, + -1, -1, -1, -1, -1, 40, -1, -1, 43, 44, + -1, 46, 47, 48, -1, 50, 51, 52, 53, 54, + -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, + -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, + 75, 76, 77, -1, -1, -1, 81, 82, 83, 84, + 85, 86, -1, 88, 89, 90, -1, 92, 93, 94, + 95, 96, 97, -1, -1, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, + -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, + 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, + -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, + -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, + 165, -1, -1, -1, 169, 170, 171, -1, 173, -1, + 175, -1, 177, 178, 179, 180, -1, 182, 183, 184, + 185, 186, 187, 188, -1, 190, 191, 192, 193, -1, + 195, 196, 197, 198, 199, 200, 201, -1, 203, -1, + 205, 206, 207, 208, 209, 210, 211, 212, -1, 214, + -1, 216, -1, -1, 219, -1, 221, 222, 223, 224, + 225, 226, -1, -1, 229, -1, 231, -1, -1, 234, + 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, + 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 258, 259, 260, 261, -1, 263, 264, + 265, 266, 267, -1, 269, 270, -1, 272, -1, 274, + 275, 276, 277, 278, 279, -1, 281, 282, -1, -1, + 285, 286, 287, -1, -1, 290, 291, 292, -1, 294, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, -1, -1, -1, -1, 311, 312, 313, -1, + 315, 316, 317, 318, 319, 320, -1, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, + 345, 346, 347, 348, 349, 350, -1, 352, 353, -1, + 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, -1, -1, 372, 373, 374, + 375, 376, -1, 378, 379, 380, 381, -1, 383, 384, + 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, -1, + 405, 406, -1, 408, -1, 410, 411, 412, 413, 414, + -1, 416, 417, 418, -1, -1, 421, 422, 423, 424, + 425, -1, 427, 428, 429, 430, 431, 432, 433, 434, + -1, -1, 437, 438, 439, -1, 441, 442, 443, 444, + -1, 446, 447, 448, 449, 450, 451, 452, -1, 454, + -1, 456, 457, 458, 459, 460, -1, -1, 463, -1, + -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, + 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, + 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, + 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, + 71, -1, 73, 74, 75, 76, 77, -1, -1, -1, + 81, 82, 83, 84, 85, 86, -1, 88, 89, 90, + -1, 92, 93, 94, 95, 96, 97, -1, -1, 100, + 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, + 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, + 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, 147, 148, 149, 150, + 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, + 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, + 171, -1, 173, -1, 175, -1, 177, 178, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, -1, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, -1, 205, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, -1, -1, 219, -1, + 221, 222, 223, 224, 225, 226, -1, -1, 229, -1, + 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, + 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, + 261, -1, 263, 264, 265, 266, 267, -1, 269, 270, + -1, 272, -1, 274, 275, 276, 277, 278, 279, -1, + 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, + 291, 292, -1, 294, -1, 296, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, 313, -1, 315, 316, 317, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + -1, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, -1, 383, 384, 385, 386, -1, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, 402, 403, -1, 405, 406, -1, 408, -1, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, + 421, 422, 423, 424, 425, -1, 427, 428, 429, 430, + 431, 432, 433, 434, -1, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + 451, 452, -1, 454, -1, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, + 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, - 37, -1, -1, -1, -1, 42, 43, 44, -1, 46, - 47, 48, 49, 50, 51, 52, -1, 54, 55, 56, + -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, + 47, 48, -1, 50, 51, 52, 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, 75, 76, - -1, -1, 79, -1, 81, 82, 83, 84, 85, 86, - 87, 88, 89, 90, -1, 92, 93, 94, 95, 96, - 97, -1, 99, 100, 101, 102, 103, 104, 105, 106, + 77, -1, -1, -1, 81, 82, 83, 84, 85, 86, + -1, 88, 89, 90, -1, 92, 93, 94, 95, 96, + 97, -1, -1, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - 147, -1, 149, 150, 151, 152, 153, -1, -1, 156, - -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, - -1, 168, 169, 170, -1, 172, 173, -1, 175, -1, - -1, -1, 179, -1, 181, 182, 183, 184, 185, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, 198, 199, 200, -1, 202, 203, -1, 205, 206, - 207, 208, 209, 210, 211, -1, 213, -1, 215, 216, - 217, 218, 219, 220, 221, 222, 223, 224, 225, -1, - -1, 228, 229, 230, 231, -1, 233, 234, 235, 236, + 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, + -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, + -1, -1, 169, 170, 171, -1, 173, -1, 175, -1, + 177, 178, 179, 180, -1, 182, 183, 184, 185, 186, + 187, 188, -1, 190, 191, 192, 193, -1, 195, 196, + 197, 198, 199, 200, 201, -1, 203, -1, 205, 206, + 207, 208, 209, 210, 211, 212, -1, 214, -1, 216, + -1, -1, 219, -1, 221, 222, 223, 224, 225, 226, + -1, -1, 229, -1, 231, -1, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, - 257, 258, 259, 260, 261, 262, 263, 264, 265, -1, - -1, 268, 269, 270, 271, -1, -1, 274, 275, 276, - 277, 278, -1, 280, 281, -1, -1, 284, 285, 286, - -1, -1, 289, -1, 291, 292, 293, -1, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, -1, -1, - -1, -1, 309, 310, -1, 312, 313, 314, -1, 316, - 317, 318, -1, 320, 321, 322, 323, 324, 325, -1, - 327, 328, 329, 330, 331, 332, 333, 334, -1, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, -1, 350, 351, 352, 353, 354, 355, 356, + 257, 258, 259, 260, 261, -1, 263, 264, 265, 266, + 267, -1, 269, 270, -1, 272, -1, 274, 275, 276, + 277, 278, 279, -1, 281, 282, -1, -1, 285, 286, + 287, -1, -1, 290, 291, 292, -1, 294, -1, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + -1, -1, -1, -1, 311, 312, 313, -1, 315, 316, + 317, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, 335, 336, + -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, -1, 352, 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, -1, 369, 370, 371, 372, 373, 374, 375, 376, - 377, 378, 379, 380, 381, 382, 383, 384, -1, 386, - 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, - 397, 398, 399, -1, 401, -1, 403, 404, -1, 406, - 407, 408, 409, 410, 411, 412, -1, 414, 415, -1, - -1, 418, 419, -1, 421, -1, -1, 424, 425, 426, - 427, 428, 429, 430, 431, 432, -1, 434, 435, 436, - -1, 438, 439, 440, 441, -1, 443, 444, 445, 446, - 447, -1, 449, -1, 451, 452, 453, 454, 455, 456, - 457, -1, -1, 460, -1, -1, 463, 464, 465, 466, - 467, 468, 3, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 480, 481, 482, 483, -1, -1, -1, - -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, - 31, 32, 33, -1, -1, -1, 37, -1, -1, -1, - -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, - 51, 52, -1, 54, 55, 56, 57, -1, 59, 60, - 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, - 71, -1, 73, 74, 75, 76, -1, -1, 79, -1, - 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, - -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, - 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, - 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, 147, -1, 149, 150, - 151, 152, 153, -1, -1, 156, -1, 158, 159, 160, - 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, - -1, 172, 173, -1, 175, -1, -1, -1, 179, -1, - 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, 198, 199, 200, - -1, 202, 203, -1, 205, 206, 207, 208, 209, 210, - 211, -1, 213, -1, 215, 216, 217, 218, 219, 220, - 221, 222, 223, 224, 225, -1, -1, 228, 229, 230, - 231, -1, 233, 234, 235, 236, 237, 238, 239, 240, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, - 261, 262, 263, 264, 265, -1, -1, 268, 269, 270, - 271, -1, -1, 274, 275, 276, 277, 278, -1, 280, - 281, -1, -1, 284, 285, 286, -1, -1, 289, -1, - 291, 292, 293, -1, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, -1, -1, -1, -1, 309, 310, - -1, 312, 313, 314, -1, 316, 317, 318, -1, 320, - 321, 322, 323, 324, 325, -1, 327, 328, 329, 330, - 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, -1, 350, - 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, -1, 369, 370, - 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, - 381, 382, 383, 384, -1, 386, 387, 388, 389, 390, - 391, 392, 393, 394, 395, 396, 397, 398, 399, -1, - 401, -1, 403, 404, -1, 406, 407, 408, 409, 410, - 411, 412, -1, 414, 415, -1, -1, 418, 419, -1, - 421, -1, -1, 424, 425, 426, 427, 428, 429, 430, - 431, 432, -1, 434, 435, 436, -1, 438, 439, 440, - 441, -1, 443, 444, 445, 446, 447, -1, 449, -1, - 451, 452, 453, 454, 455, 456, 457, -1, -1, 460, - -1, -1, 463, 464, 465, 466, 467, 468, 3, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 480, - 481, 482, 483, -1, -1, -1, -1, 22, 23, 24, - 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, - -1, -1, 37, -1, -1, -1, -1, 42, 43, 44, - -1, 46, 47, 48, 49, 50, 51, 52, -1, 54, - 55, 56, 57, -1, 59, 60, 61, 62, 63, 64, - -1, -1, 67, 68, 69, 70, 71, -1, 73, 74, - 75, 76, -1, -1, 79, -1, 81, 82, 83, 84, - 85, 86, 87, 88, 89, 90, -1, 92, 93, 94, - 95, 96, 97, -1, 99, 100, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, -1, 114, - -1, 116, 117, 118, 119, 120, 121, -1, 123, 124, - 125, 126, 127, -1, -1, 130, 131, 132, 133, 134, - -1, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, 147, -1, 149, 150, 151, 152, 153, -1, - -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, - 165, 166, -1, 168, 169, 170, -1, 172, 173, -1, - 175, -1, -1, -1, 179, -1, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, 198, 199, 200, -1, 202, 203, -1, - 205, 206, 207, 208, 209, 210, 211, -1, 213, -1, - 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, - 225, -1, -1, 228, 229, 230, 231, -1, 233, 234, - 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, - 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, - 265, -1, -1, 268, 269, 270, 271, -1, -1, 274, - 275, 276, 277, 278, -1, 280, 281, -1, -1, 284, - 285, 286, -1, -1, 289, -1, 291, 292, 293, -1, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - -1, -1, -1, -1, 309, 310, -1, 312, 313, 314, - -1, 316, 317, 318, -1, 320, 321, 322, 323, 324, - 325, -1, 327, 328, 329, 330, 331, 332, 333, 334, - -1, 336, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, -1, 350, 351, 352, 353, 354, - 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, -1, 369, 370, 371, 372, 373, 374, - 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, - -1, 386, 387, 388, 389, 390, 391, 392, 393, 394, - 395, 396, 397, 398, 399, -1, 401, -1, 403, 404, - -1, 406, 407, 408, 409, 410, 411, 412, -1, 414, - 415, -1, -1, 418, 419, -1, 421, -1, -1, 424, - 425, 426, 427, 428, 429, 430, 431, 432, -1, 434, - 435, 436, -1, 438, 439, 440, 441, -1, 443, 444, - 445, 446, 447, -1, 449, -1, 451, 452, 453, 454, - 455, 456, 457, -1, -1, 460, -1, -1, 463, 464, - 465, 466, 467, 468, 3, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 480, 481, 482, 483, -1, + 367, 368, 369, -1, -1, 372, 373, 374, 375, 376, + 377, 378, 379, 380, 381, -1, 383, 384, 385, 386, + -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 397, 398, 399, 400, 401, 402, 403, -1, 405, 406, + -1, 408, -1, 410, 411, 412, 413, 414, -1, 416, + 417, 418, -1, -1, 421, 422, 423, 424, 425, -1, + 427, 428, 429, 430, 431, 432, 433, 434, -1, -1, + 437, 438, 439, -1, 441, 442, 443, 444, -1, 446, + 447, 448, 449, 450, 451, 452, -1, 454, -1, 456, + 457, 458, 459, 460, -1, -1, 463, -1, -1, 466, + 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, + 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, + 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, + 53, 54, -1, 56, 57, -1, 59, 60, 61, 62, + 63, 64, -1, -1, 67, 68, 69, 70, 71, -1, + 73, 74, 75, 76, 77, -1, -1, -1, 81, 82, + 83, 84, 85, 86, -1, 88, 89, 90, -1, 92, + 93, 94, 95, 96, 97, -1, -1, 100, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + -1, 114, -1, 116, 117, 118, 119, 120, 121, -1, + 123, 124, 125, 126, 127, -1, -1, 130, 131, 132, + 133, 134, -1, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, + 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, + 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, + 173, -1, 175, -1, 177, 178, 179, 180, -1, 182, + 183, 184, 185, 186, 187, 188, -1, 190, 191, 192, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, -1, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, -1, -1, 219, -1, 221, 222, + 223, 224, 225, 226, -1, -1, 229, -1, 231, -1, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, + 253, 254, 255, 256, 257, 258, 259, 260, 261, -1, + 263, 264, 265, 266, 267, -1, 269, 270, -1, 272, + -1, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, + -1, 294, -1, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, -1, -1, -1, -1, 311, 312, + 313, -1, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, + 353, -1, 355, 356, 357, 358, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, -1, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, -1, 408, -1, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, -1, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, -1, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, -1, 456, 457, 458, 459, 460, -1, -1, + 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, + 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, 37, -1, -1, -1, -1, 42, 43, 44, -1, 46, 47, 48, @@ -17760,38 +17740,179 @@ static const yytype_int16 yycheck[] = -1, 140, 141, 142, -1, 144, 145, 146, 147, -1, 149, 150, 151, 152, 153, -1, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, - 169, 170, 171, 172, 173, -1, 175, -1, -1, -1, - 179, -1, 181, 182, 183, 184, 185, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, 198, - 199, 200, -1, 202, 203, -1, 205, 206, 207, 208, - 209, 210, 211, -1, 213, -1, 215, 216, 217, 218, - 219, 220, 221, 222, 223, 224, 225, -1, -1, 228, - 229, 230, 231, -1, 233, 234, 235, 236, 237, 238, + 169, 170, -1, 172, 173, -1, 175, -1, -1, -1, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, 204, -1, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, -1, -1, + 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, 261, 262, 263, 264, 265, -1, -1, 268, - 269, 270, 271, -1, -1, 274, 275, 276, 277, 278, - -1, 280, 281, -1, -1, 284, 285, 286, -1, -1, - 289, -1, 291, 292, 293, -1, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, -1, -1, -1, -1, - 309, 310, -1, 312, 313, 314, -1, 316, 317, 318, - -1, 320, 321, 322, 323, 324, 325, -1, 327, 328, - 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, + 259, 260, 261, 262, 263, 264, 265, 266, -1, -1, + 269, 270, 271, 272, -1, -1, 275, 276, 277, 278, + 279, -1, 281, 282, -1, -1, 285, 286, 287, -1, + -1, 290, 291, -1, 293, 294, 295, -1, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, -1, 314, 315, 316, -1, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, - -1, 350, 351, 352, 353, 354, 355, -1, 357, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, -1, - 369, 370, 371, 372, 373, 374, 375, -1, 377, 378, - 379, 380, 381, 382, 383, 384, -1, 386, 387, 388, + 349, 350, -1, 352, 353, 354, 355, 356, 357, 358, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, 371, 372, 373, 374, 375, 376, 377, 378, + 379, 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, - 399, -1, 401, -1, 403, 404, -1, 406, 407, 408, - 409, 410, 411, 412, -1, 414, 415, -1, -1, 418, - 419, -1, 421, -1, -1, 424, 425, 426, 427, 428, - 429, 430, 431, -1, -1, 434, 435, 436, -1, 438, - 439, 440, 441, -1, 443, 444, 445, 446, 447, -1, - 449, -1, 451, 452, 453, 454, 455, 456, 457, -1, - -1, 460, -1, -1, 463, 464, 465, 466, 467, 468, + 399, 400, 401, -1, 403, -1, 405, 406, -1, 408, + 409, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, -1, 424, -1, -1, 427, 428, + 429, 430, 431, 432, 433, 434, 435, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, -1, 452, -1, 454, 455, 456, 457, 458, + 459, 460, -1, -1, 463, -1, -1, 466, 467, 468, + 469, 470, 471, 3, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 483, 484, 485, 486, -1, -1, + -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, + -1, 31, 32, 33, -1, -1, -1, 37, -1, -1, + -1, -1, 42, 43, 44, -1, 46, 47, 48, 49, + 50, 51, 52, -1, 54, 55, 56, 57, -1, 59, + 60, 61, 62, 63, 64, -1, -1, 67, 68, 69, + 70, 71, -1, 73, 74, 75, 76, -1, -1, 79, + -1, 81, 82, 83, 84, 85, 86, 87, 88, 89, + 90, -1, 92, 93, 94, 95, 96, 97, -1, 99, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, -1, 114, -1, 116, 117, 118, 119, + 120, 121, -1, 123, 124, 125, 126, 127, -1, -1, + 130, 131, 132, 133, 134, -1, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, 147, -1, 149, + 150, 151, 152, 153, -1, -1, 156, -1, 158, 159, + 160, 161, -1, 163, -1, 165, 166, -1, 168, 169, + 170, -1, 172, 173, -1, 175, -1, -1, -1, 179, + 180, -1, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, -1, 195, 196, 197, 198, 199, + 200, 201, -1, 203, 204, -1, 206, 207, 208, 209, + 210, 211, 212, -1, 214, -1, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, -1, -1, 229, + 230, 231, 232, -1, 234, 235, 236, 237, 238, 239, + 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, + 260, 261, 262, 263, 264, 265, 266, -1, -1, 269, + 270, 271, 272, -1, -1, 275, 276, 277, 278, 279, + -1, 281, 282, -1, -1, 285, 286, 287, -1, -1, + 290, 291, -1, 293, 294, 295, -1, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, -1, -1, -1, + -1, 311, 312, -1, 314, 315, 316, -1, 318, 319, + 320, -1, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, 335, 336, -1, 338, 339, + 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, + 350, -1, 352, 353, 354, 355, 356, 357, 358, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + -1, 371, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, -1, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, + 400, 401, -1, 403, -1, 405, 406, -1, 408, 409, + 410, 411, 412, 413, 414, -1, 416, 417, 418, -1, + -1, 421, 422, -1, 424, -1, -1, 427, 428, 429, + 430, 431, 432, 433, 434, 435, -1, 437, 438, 439, + -1, 441, 442, 443, 444, -1, 446, 447, 448, 449, + 450, -1, 452, -1, 454, 455, 456, 457, 458, 459, + 460, -1, -1, 463, -1, -1, 466, 467, 468, 469, + 470, 471, 3, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 483, 484, 485, 486, -1, -1, -1, + -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, + 31, 32, 33, -1, -1, -1, 37, -1, -1, -1, + -1, 42, 43, 44, -1, 46, 47, 48, 49, 50, + 51, 52, -1, 54, 55, 56, 57, -1, 59, 60, + 61, 62, 63, 64, -1, -1, 67, 68, 69, 70, + 71, -1, 73, 74, 75, 76, -1, -1, 79, -1, + 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, + -1, 92, 93, 94, 95, 96, 97, -1, 99, 100, + 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, -1, 114, -1, 116, 117, 118, 119, 120, + 121, -1, 123, 124, 125, 126, 127, -1, -1, 130, + 131, 132, 133, 134, -1, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, 147, -1, 149, 150, + 151, 152, 153, -1, -1, 156, -1, 158, 159, 160, + 161, -1, 163, -1, 165, 166, -1, 168, 169, 170, + -1, 172, 173, -1, 175, -1, -1, -1, 179, 180, + -1, 182, 183, 184, 185, 186, 187, 188, 189, 190, + 191, 192, 193, -1, 195, 196, 197, 198, 199, 200, + 201, -1, 203, 204, -1, 206, 207, 208, 209, 210, + 211, 212, -1, 214, -1, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, -1, -1, 229, 230, + 231, 232, -1, 234, 235, 236, 237, 238, 239, 240, + 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, + 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, + 261, 262, 263, 264, 265, 266, -1, -1, 269, 270, + 271, 272, -1, -1, 275, 276, 277, 278, 279, -1, + 281, 282, -1, -1, 285, 286, 287, -1, -1, 290, + 291, -1, 293, 294, 295, -1, 297, 298, 299, 300, + 301, 302, 303, 304, 305, 306, -1, -1, -1, -1, + 311, 312, -1, 314, 315, 316, -1, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, 335, 336, -1, 338, 339, 340, + 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, + -1, 352, 353, 354, 355, 356, 357, 358, 359, 360, + 361, 362, 363, 364, 365, 366, 367, 368, 369, -1, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, + 381, 382, 383, 384, 385, 386, -1, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, + 401, -1, 403, -1, 405, 406, -1, 408, 409, 410, + 411, 412, 413, 414, -1, 416, 417, 418, -1, -1, + 421, 422, -1, 424, -1, -1, 427, 428, 429, 430, + 431, 432, 433, 434, 435, -1, 437, 438, 439, -1, + 441, 442, 443, 444, -1, 446, 447, 448, 449, 450, + -1, 452, -1, 454, 455, 456, 457, 458, 459, 460, + -1, -1, 463, -1, -1, 466, 467, 468, 469, 470, + 471, 3, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 483, 484, 485, 486, -1, -1, -1, -1, + 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, + 32, 33, -1, -1, -1, 37, -1, -1, -1, -1, + 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, + 52, -1, 54, 55, 56, 57, -1, 59, 60, 61, + 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, + -1, 73, 74, 75, 76, -1, -1, 79, -1, 81, + 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, + 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, + -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, + 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, 147, -1, 149, 150, 151, + 152, 153, -1, -1, 156, -1, 158, 159, 160, 161, + -1, 163, -1, 165, 166, -1, 168, 169, 170, 171, + 172, 173, -1, 175, -1, -1, -1, 179, 180, -1, + 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, + 192, 193, -1, 195, 196, 197, 198, 199, 200, 201, + -1, 203, 204, -1, 206, 207, 208, 209, 210, 211, + 212, -1, 214, -1, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, -1, -1, 229, 230, 231, + 232, -1, 234, 235, 236, 237, 238, 239, 240, 241, + 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, + 262, 263, 264, 265, 266, -1, -1, 269, 270, 271, + 272, -1, -1, 275, 276, 277, 278, 279, -1, 281, + 282, -1, -1, 285, 286, 287, -1, -1, 290, 291, + -1, 293, 294, 295, -1, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, -1, -1, -1, -1, 311, + 312, -1, 314, 315, 316, -1, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, 335, 336, -1, 338, 339, 340, 341, + 342, 343, 344, 345, 346, 347, 348, 349, 350, -1, + 352, 353, 354, 355, 356, 357, -1, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, -1, 371, + 372, 373, 374, 375, 376, 377, -1, 379, 380, 381, + 382, 383, 384, 385, 386, -1, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, + -1, 403, -1, 405, 406, -1, 408, 409, 410, 411, + 412, 413, 414, -1, 416, 417, 418, -1, -1, 421, + 422, -1, 424, -1, -1, 427, 428, 429, 430, 431, + 432, 433, 434, -1, -1, 437, 438, 439, -1, 441, + 442, 443, 444, -1, 446, 447, 448, 449, 450, -1, + 452, -1, 454, 455, 456, 457, 458, 459, 460, -1, + -1, 463, -1, -1, 466, 467, 468, 469, 470, 471, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 480, 481, 482, 483, -1, -1, -1, -1, 22, + -1, 483, 484, 485, 486, -1, -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, 32, 33, -1, -1, -1, -1, -1, -1, -1, -1, -1, 43, 44, -1, 46, 47, 48, -1, 50, 51, 52, @@ -17807,133 +17928,134 @@ static const yytype_int16 yycheck[] = -1, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, -1, 156, -1, 158, 159, 160, 161, -1, 163, -1, 165, -1, -1, -1, 169, 170, 171, -1, - 173, -1, 175, -1, 177, 178, 179, -1, 181, 182, - 183, 184, 185, 186, 187, -1, 189, 190, 191, 192, - -1, 194, 195, 196, 197, 198, 199, 200, -1, 202, - -1, 204, 205, 206, 207, 208, 209, 210, 211, -1, - 213, -1, 215, -1, -1, 218, -1, 220, 221, 222, - 223, 224, 225, -1, -1, 228, -1, 230, -1, -1, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 173, -1, 175, -1, 177, 178, -1, 180, -1, 182, + 183, 184, 185, 186, 187, 188, -1, 190, 191, 192, + 193, -1, 195, 196, 197, 198, 199, 200, 201, -1, + 203, -1, 205, 206, 207, 208, 209, 210, 211, 212, + -1, 214, -1, 216, -1, -1, 219, -1, 221, 222, + 223, 224, 225, 226, -1, -1, 229, -1, 231, -1, + -1, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 253, 254, 255, 256, 257, 258, 259, 260, -1, 262, - 263, 264, 265, 266, -1, 268, 269, -1, 271, -1, - 273, 274, 275, 276, 277, 278, -1, 280, 281, -1, - -1, 284, 285, 286, -1, -1, 289, 290, -1, 292, - -1, 294, 295, 296, 297, 298, 299, 300, -1, 302, - 303, 304, -1, -1, -1, -1, 309, 310, 311, -1, - 313, 314, 315, 316, 317, 318, -1, 320, 321, 322, - 323, 324, 325, -1, 327, -1, 329, 330, 331, 332, - 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, - 343, 344, 345, 346, 347, 348, -1, 350, 351, -1, - 353, 354, 355, 356, -1, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, -1, -1, 370, 371, 372, - 373, 374, 375, 376, 377, 378, 379, -1, 381, 382, - 383, 384, -1, 386, 387, 388, 389, 390, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, -1, - 403, 404, -1, 406, -1, 408, 409, 410, 411, 412, - -1, 414, 415, -1, -1, 418, 419, 420, 421, 422, - -1, 424, 425, 426, 427, 428, 429, 430, 431, -1, - -1, 434, 435, 436, -1, 438, 439, 440, 441, -1, - 443, 444, 445, 446, 447, 448, 449, -1, 451, -1, - 453, 454, 455, 456, 457, -1, -1, 460, -1, -1, - 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, + 253, 254, 255, 256, 257, 258, 259, 260, 261, -1, + 263, 264, 265, 266, 267, -1, 269, 270, -1, 272, + -1, 274, 275, 276, 277, 278, 279, -1, 281, 282, + -1, -1, 285, 286, 287, -1, -1, 290, 291, 292, + -1, 294, -1, 296, 297, 298, 299, 300, 301, 302, + -1, 304, 305, 306, -1, -1, -1, -1, 311, 312, + 313, -1, 315, 316, 317, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, -1, 331, 332, + 333, 334, 335, 336, -1, 338, 339, 340, 341, 342, + 343, 344, 345, 346, 347, 348, 349, 350, -1, 352, + 353, -1, 355, 356, 357, 358, -1, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, -1, -1, 372, + 373, 374, 375, 376, 377, 378, 379, 380, 381, -1, + 383, 384, 385, 386, -1, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, + 403, -1, 405, 406, -1, 408, -1, 410, 411, 412, + 413, 414, -1, 416, 417, 418, -1, -1, 421, 422, + 423, 424, 425, -1, 427, 428, 429, 430, 431, 432, + 433, 434, -1, -1, 437, 438, 439, -1, 441, 442, + 443, 444, -1, 446, 447, 448, 449, 450, 451, 452, + -1, 454, -1, 456, 457, 458, 459, 460, -1, -1, + 463, -1, -1, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 3, -1, -1, -1, -1, -1, -1, -1, -1, + 483, 484, 485, 486, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 22, 23, 24, 25, 26, 27, 28, 29, -1, 31, - 32, 33, -1, -1, -1, 37, -1, -1, -1, -1, - 42, 43, 44, -1, 46, 47, 48, 49, 50, 51, - 52, -1, 54, 55, 56, 57, -1, 59, 60, 61, - 62, 63, 64, -1, -1, 67, 68, 69, 70, 71, - -1, 73, 74, 75, 76, -1, -1, 79, -1, 81, - 82, 83, 84, 85, 86, 87, 88, 89, 90, -1, - 92, 93, 94, 95, 96, 97, -1, 99, 100, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, -1, 114, -1, 116, 117, 118, 119, 120, 121, - -1, 123, 124, 125, 126, 127, -1, -1, 130, 131, - 132, 133, 134, -1, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, 147, -1, 149, 150, 151, - 152, 153, -1, -1, 156, -1, 158, 159, 160, 161, - -1, 163, -1, 165, 166, -1, 168, 169, 170, -1, - 172, 173, -1, 175, -1, -1, -1, 179, -1, 181, - 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, 198, 199, 200, -1, - 202, 203, -1, 205, 206, 207, 208, 209, 210, 211, - -1, 213, -1, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, -1, -1, 228, 229, 230, 231, - -1, 233, 234, 235, 236, 237, 238, 239, 240, -1, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, - 262, 263, 264, 265, -1, -1, 268, 269, 270, 271, - -1, -1, 274, 275, 276, 277, 278, -1, 280, 281, - -1, -1, 284, 285, 286, -1, -1, 289, -1, 291, - 292, 293, -1, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, -1, -1, -1, -1, 309, 310, -1, - 312, 313, 314, -1, 316, 317, 318, -1, 320, 321, - 322, 323, 324, 325, -1, 327, 328, 329, 330, 331, - 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, - 342, 343, 344, 345, 346, 347, 348, -1, 350, 351, - 352, 353, 354, 355, -1, 357, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, -1, 369, 370, 371, - 372, 373, 374, 375, -1, 377, 378, 379, 380, 381, - 382, 383, 384, -1, 386, 387, 388, 389, 390, 391, - 392, 393, 394, 395, 396, 397, -1, 399, -1, 401, - -1, 403, 404, -1, 406, 407, 408, 409, 410, 411, - 412, -1, 414, 415, -1, -1, 418, 419, -1, 421, - -1, -1, 424, 425, 426, 427, 428, 429, 430, 431, - -1, -1, 434, 435, 436, -1, 438, 439, 440, 441, - -1, 443, 444, 445, 446, 447, -1, 449, -1, 451, - 452, 453, 454, 455, 456, 457, -1, 22, 460, -1, - -1, 463, 464, 465, 466, 467, 468, 32, -1, 34, - 35, -1, -1, -1, 22, -1, -1, -1, 480, 481, - 482, 483, -1, -1, 32, -1, -1, 52, -1, -1, - -1, -1, -1, -1, -1, -1, 61, -1, -1, -1, - -1, -1, -1, -1, 52, -1, -1, -1, 73, -1, - -1, -1, -1, 61, -1, -1, -1, -1, -1, 84, - -1, -1, -1, -1, -1, 73, -1, -1, -1, -1, - -1, 96, -1, 98, -1, -1, 84, -1, -1, -1, - -1, -1, -1, -1, 109, -1, -1, -1, 96, -1, - 98, -1, -1, -1, -1, -1, -1, -1, 123, -1, - -1, 109, -1, -1, -1, -1, -1, -1, 133, -1, - -1, -1, -1, -1, 139, 123, -1, -1, -1, -1, - -1, -1, 147, -1, -1, 133, -1, -1, -1, -1, - -1, 139, -1, -1, -1, -1, -1, -1, 163, 147, - -1, -1, 167, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 163, -1, -1, -1, 167, + -1, -1, -1, 22, 23, 24, 25, 26, 27, 28, + 29, -1, 31, 32, 33, -1, -1, -1, 37, -1, + -1, -1, -1, 42, 43, 44, -1, 46, 47, 48, + 49, 50, 51, 52, -1, 54, 55, 56, 57, -1, + 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, + 69, 70, 71, -1, 73, 74, 75, 76, -1, -1, + 79, -1, 81, 82, 83, 84, 85, 86, 87, 88, + 89, 90, -1, 92, 93, 94, 95, 96, 97, -1, + 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, + 109, 110, 111, 112, -1, 114, -1, 116, 117, 118, + 119, 120, 121, -1, 123, 124, 125, 126, 127, -1, + -1, 130, 131, 132, 133, 134, -1, 136, 137, 138, + -1, 140, 141, 142, -1, 144, 145, 146, 147, -1, + 149, 150, 151, 152, 153, -1, -1, 156, -1, 158, + 159, 160, 161, -1, 163, -1, 165, 166, -1, 168, + 169, 170, -1, 172, 173, -1, 175, -1, -1, -1, + 179, 180, -1, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, -1, 195, 196, 197, 198, + 199, 200, 201, -1, 203, 204, -1, 206, 207, 208, + 209, 210, 211, 212, -1, 214, -1, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, -1, -1, + 229, 230, 231, 232, -1, 234, 235, 236, 237, 238, + 239, 240, 241, -1, 243, 244, 245, 246, 247, 248, + 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, + 259, 260, 261, 262, 263, 264, 265, 266, -1, -1, + 269, 270, 271, 272, -1, -1, 275, 276, 277, 278, + 279, -1, 281, 282, -1, -1, 285, 286, 287, -1, + -1, 290, 291, -1, 293, 294, 295, -1, 297, 298, + 299, 300, 301, 302, 303, 304, 305, 306, -1, -1, + -1, -1, 311, 312, -1, 314, 315, 316, -1, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, 335, 336, -1, 338, + 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, + 349, 350, -1, 352, 353, 354, 355, 356, 357, -1, + 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, + 369, -1, 371, 372, 373, 374, 375, 376, 377, -1, + 379, 380, 381, 382, 383, 384, 385, 386, -1, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, + 399, -1, 401, -1, 403, -1, 405, 406, -1, 408, + 409, 410, 411, 412, 413, 414, -1, 416, 417, 418, + -1, -1, 421, 422, -1, 424, -1, -1, 427, 428, + 429, 430, 431, 432, 433, 434, -1, -1, 437, 438, + 439, -1, 441, 442, 443, 444, -1, 446, 447, 448, + 449, 450, -1, 452, -1, 454, 455, 456, 457, 458, + 459, 460, -1, 22, 463, -1, -1, 466, 467, 468, + 469, 470, 471, 32, -1, 34, 35, -1, -1, -1, + 22, -1, -1, -1, 483, 484, 485, 486, -1, -1, + 32, -1, -1, 52, -1, -1, -1, -1, -1, -1, + -1, -1, 61, -1, -1, -1, -1, -1, -1, -1, + 52, -1, -1, -1, 73, -1, -1, -1, -1, 61, + -1, -1, -1, -1, -1, 84, -1, -1, -1, -1, + -1, 73, -1, -1, -1, -1, -1, 96, -1, 98, + -1, -1, 84, -1, -1, -1, -1, -1, -1, -1, + 109, -1, -1, -1, 96, -1, 98, -1, -1, -1, + -1, -1, -1, -1, 123, -1, -1, 109, -1, -1, + -1, -1, -1, -1, 133, -1, -1, -1, -1, -1, + 139, 123, -1, -1, -1, -1, -1, -1, 147, -1, + -1, 133, -1, -1, -1, -1, -1, 139, -1, -1, + -1, -1, -1, -1, 163, 147, -1, -1, 167, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 163, -1, -1, -1, 167, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 208, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 208, -1, -1, -1, -1, -1, -1, -1, -1, 234, + 209, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 209, -1, -1, + -1, -1, -1, -1, -1, -1, 235, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 234, -1, -1, -1, + -1, -1, -1, 235, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 307, 308, + 309, -1, -1, -1, -1, -1, 315, -1, -1, 318, + -1, -1, -1, -1, -1, 307, 308, 309, -1, -1, + -1, -1, -1, 315, -1, -1, 318, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 347, -1, + -1, -1, -1, -1, -1, -1, -1, 356, -1, -1, + -1, -1, -1, -1, -1, 347, -1, -1, -1, -1, + -1, 370, -1, -1, 356, -1, -1, -1, 377, -1, + -1, -1, 381, -1, -1, -1, -1, -1, 370, -1, + -1, -1, 391, -1, -1, 377, -1, -1, -1, 381, + -1, -1, -1, -1, 403, -1, -1, -1, 407, 391, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 305, 306, 307, -1, -1, -1, -1, -1, 313, -1, - -1, 316, -1, -1, -1, -1, -1, 305, 306, 307, - -1, -1, -1, -1, -1, 313, -1, -1, 316, -1, + -1, 403, -1, -1, -1, 407, -1, -1, 427, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 345, -1, -1, -1, -1, -1, -1, -1, -1, 354, - -1, -1, -1, -1, -1, -1, -1, 345, -1, -1, - -1, -1, -1, 368, -1, -1, 354, -1, -1, -1, - 375, -1, -1, -1, 379, -1, -1, -1, -1, -1, - 368, -1, -1, -1, 389, -1, -1, 375, -1, -1, - -1, 379, -1, -1, -1, -1, 401, -1, -1, -1, - 405, 389, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 401, -1, -1, -1, 405, -1, 424, + -1, 440, -1, -1, -1, 427, -1, 446, -1, -1, + -1, -1, 451, -1, -1, -1, 455, -1, 440, -1, + -1, -1, -1, -1, 446, -1, 465, -1, -1, 451, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 437, -1, -1, -1, 424, -1, 443, -1, - -1, -1, -1, 448, -1, -1, -1, 452, -1, 437, - -1, -1, -1, -1, -1, 443, -1, 462, -1, -1, - 448, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 462, -1, -1, -1, -1, -1, - -1, 486, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 500, -1, -1, 486, -1, + -1, -1, -1, 465, -1, -1, -1, -1, -1, -1, + 489, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 503, -1, -1, 489, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 500 + -1, 503 }; /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing @@ -17942,15 +18064,15 @@ static const yytype_uint16 yystos[] = { 0, 22, 32, 34, 35, 47, 52, 61, 73, 84, 96, 98, 109, 123, 124, 133, 139, 147, 149, 150, - 163, 167, 192, 208, 234, 305, 306, 307, 313, 316, - 345, 354, 368, 375, 379, 389, 401, 405, 424, 437, - 440, 443, 448, 462, 486, 500, 512, 513, 514, 515, - 526, 535, 537, 542, 558, 561, 562, 564, 568, 575, - 577, 578, 626, 633, 636, 637, 654, 655, 656, 657, - 659, 661, 662, 666, 719, 720, 880, 883, 886, 893, - 894, 896, 897, 898, 905, 909, 915, 918, 923, 927, - 928, 929, 932, 935, 936, 940, 941, 943, 418, 465, - 576, 197, 361, 370, 405, 454, 106, 930, 576, 3, + 163, 167, 193, 209, 235, 307, 308, 309, 315, 318, + 347, 356, 370, 377, 381, 391, 403, 407, 427, 440, + 443, 446, 451, 465, 489, 503, 515, 516, 517, 518, + 529, 538, 540, 545, 561, 564, 565, 567, 571, 578, + 580, 581, 629, 636, 639, 640, 657, 658, 659, 660, + 662, 664, 665, 669, 722, 723, 884, 887, 890, 897, + 898, 900, 901, 902, 909, 913, 919, 922, 927, 931, + 932, 933, 936, 939, 940, 944, 945, 947, 421, 468, + 579, 198, 363, 372, 407, 457, 106, 934, 579, 3, 22, 23, 24, 25, 26, 27, 28, 29, 31, 32, 33, 42, 43, 44, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 59, 60, 61, 62, @@ -17963,323 +18085,325 @@ static const yytype_uint16 yystos[] = 136, 137, 138, 140, 141, 142, 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, 156, 158, 159, 160, 161, 163, 165, 166, 168, 169, 170, 171, 172, - 173, 175, 177, 178, 179, 181, 182, 183, 184, 185, - 186, 187, 188, 189, 190, 191, 192, 194, 195, 196, - 197, 198, 199, 200, 202, 203, 204, 205, 206, 207, - 208, 209, 210, 211, 213, 215, 216, 217, 218, 219, - 220, 221, 222, 223, 224, 225, 228, 229, 230, 231, - 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, + 173, 175, 177, 178, 179, 180, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, 195, 196, + 197, 198, 199, 200, 201, 203, 204, 205, 206, 207, + 208, 209, 210, 211, 212, 214, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, 229, 230, 231, + 232, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, - 263, 264, 265, 266, 268, 269, 270, 271, 273, 274, - 275, 276, 277, 278, 280, 281, 284, 285, 286, 289, + 263, 264, 265, 266, 267, 269, 270, 271, 272, 274, + 275, 276, 277, 278, 279, 281, 282, 285, 286, 287, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 309, 310, 311, 312, 313, - 314, 315, 316, 317, 318, 320, 321, 322, 323, 324, - 325, 327, 328, 329, 330, 331, 332, 333, 334, 336, - 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, - 347, 348, 350, 351, 352, 353, 354, 355, 356, 357, + 300, 301, 302, 303, 304, 305, 306, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, 322, 323, 324, + 325, 326, 327, 329, 330, 331, 332, 333, 334, 335, + 336, 338, 339, 340, 341, 342, 343, 344, 345, 346, + 347, 348, 349, 350, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, - 380, 381, 382, 383, 384, 386, 387, 388, 389, 390, + 368, 369, 372, 373, 374, 375, 376, 377, 378, 379, + 380, 381, 382, 383, 384, 385, 386, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, - 401, 403, 404, 406, 407, 408, 409, 410, 411, 412, - 414, 415, 418, 419, 420, 421, 422, 424, 425, 426, - 427, 428, 429, 430, 431, 434, 435, 436, 438, 439, - 440, 441, 443, 444, 445, 446, 447, 448, 449, 451, - 452, 453, 454, 455, 456, 457, 460, 463, 464, 465, + 401, 402, 403, 405, 406, 408, 409, 410, 411, 412, + 413, 414, 416, 417, 418, 421, 422, 423, 424, 425, + 427, 428, 429, 430, 431, 432, 433, 434, 437, 438, + 439, 441, 442, 443, 444, 446, 447, 448, 449, 450, + 451, 452, 454, 455, 456, 457, 458, 459, 460, 463, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 528, 790, - 869, 873, 946, 947, 948, 3, 171, 241, 398, 528, - 895, 946, 576, 55, 500, 649, 173, 235, 287, 361, - 409, 411, 427, 433, 436, 624, 892, 5, 30, 316, - 528, 529, 868, 106, 930, 24, 79, 95, 142, 152, - 164, 169, 197, 240, 244, 310, 325, 358, 361, 370, - 373, 391, 405, 412, 421, 427, 454, 627, 628, 631, - 576, 868, 452, 500, 515, 526, 535, 537, 558, 561, - 562, 564, 568, 575, 578, 626, 633, 636, 637, 654, - 880, 883, 886, 893, 894, 901, 905, 909, 915, 918, - 923, 932, 935, 940, 941, 943, 106, 73, 208, 66, - 77, 79, 154, 226, 273, 283, 294, 311, 357, 400, - 420, 422, 426, 448, 500, 527, 528, 529, 655, 720, - 722, 724, 734, 741, 742, 790, 792, 793, 106, 5, - 528, 530, 916, 916, 528, 868, 30, 173, 235, 374, - 414, 418, 528, 933, 934, 938, 576, 30, 128, 675, - 676, 173, 235, 361, 374, 414, 910, 911, 938, 576, - 528, 654, 666, 939, 528, 741, 405, 672, 527, 168, - 500, 920, 500, 333, 667, 668, 868, 667, 655, 656, - 0, 503, 118, 207, 439, 143, 212, 288, 432, 678, - 679, 724, 724, 655, 657, 659, 504, 452, 899, 30, - 414, 418, 654, 939, 186, 527, 868, 186, 527, 186, - 741, 186, 527, 530, 498, 502, 531, 532, 500, 654, - 527, 409, 411, 409, 411, 343, 186, 527, 527, 169, - 240, 333, 370, 405, 454, 634, 197, 30, 868, 186, - 3, 246, 421, 105, 405, 405, 454, 364, 3, 37, - 42, 49, 54, 55, 56, 57, 69, 70, 79, 81, - 87, 99, 110, 111, 132, 160, 166, 168, 172, 186, - 188, 203, 210, 211, 213, 216, 217, 219, 229, 231, - 241, 260, 261, 262, 270, 275, 291, 293, 312, 330, - 352, 356, 369, 376, 380, 383, 398, 407, 414, 415, - 426, 432, 449, 452, 632, 752, 754, 756, 758, 760, - 762, 764, 765, 766, 768, 769, 770, 772, 773, 874, - 946, 949, 186, 528, 629, 875, 186, 865, 868, 186, - 868, 500, 567, 901, 3, 37, 42, 49, 53, 54, - 55, 56, 57, 69, 70, 77, 79, 81, 87, 99, - 110, 111, 148, 154, 160, 166, 168, 171, 172, 177, - 178, 188, 203, 204, 210, 211, 213, 216, 217, 219, - 229, 231, 241, 260, 261, 262, 266, 270, 273, 275, - 290, 291, 293, 294, 311, 312, 315, 330, 352, 356, - 369, 376, 380, 383, 398, 400, 407, 414, 415, 420, - 422, 426, 448, 449, 452, 469, 470, 471, 472, 473, - 474, 475, 476, 477, 478, 479, 902, 904, 905, 907, - 908, 946, 950, 899, 528, 530, 895, 916, 500, 500, - 162, 500, 655, 742, 500, 500, 527, 500, 500, 167, - 500, 500, 500, 500, 655, 720, 724, 734, 493, 531, - 40, 528, 735, 736, 735, 368, 504, 658, 37, 42, - 99, 168, 203, 219, 229, 261, 305, 312, 352, 369, - 437, 738, 736, 40, 528, 735, 737, 486, 746, 530, - 167, 489, 500, 500, 881, 934, 934, 934, 483, 218, - 502, 282, 4, 6, 7, 8, 9, 10, 39, 54, - 56, 57, 65, 69, 70, 81, 110, 111, 113, 148, - 155, 160, 177, 178, 210, 211, 213, 241, 260, 262, - 267, 272, 275, 284, 330, 356, 383, 414, 415, 423, - 449, 484, 491, 492, 493, 498, 500, 505, 506, 507, - 508, 528, 530, 655, 709, 757, 760, 763, 764, 765, - 767, 768, 769, 772, 773, 784, 786, 787, 788, 789, - 790, 791, 793, 794, 808, 809, 820, 842, 847, 855, - 856, 857, 869, 870, 871, 854, 856, 910, 910, 530, - 910, 483, 167, 416, 489, 502, 531, 741, 924, 3, - 166, 168, 452, 905, 919, 921, 166, 922, 784, 826, - 827, 667, 504, 500, 877, 501, 501, 514, 167, 287, - 545, 924, 30, 128, 673, 673, 59, 673, 157, 162, - 232, 279, 684, 686, 687, 712, 714, 715, 716, 176, - 282, 442, 282, 678, 679, 500, 527, 406, 937, 483, - 218, 148, 26, 32, 133, 286, 341, 345, 375, 445, - 520, 523, 524, 341, 148, 40, 60, 104, 196, 245, - 253, 265, 296, 341, 347, 370, 375, 389, 523, 569, - 572, 148, 341, 375, 523, 148, 341, 375, 523, 40, - 931, 784, 848, 3, 30, 34, 35, 36, 38, 39, - 40, 41, 45, 58, 65, 66, 72, 78, 80, 91, - 98, 113, 115, 122, 128, 129, 135, 139, 143, 155, - 157, 162, 164, 167, 174, 176, 180, 193, 201, 212, - 214, 226, 227, 232, 267, 272, 279, 282, 283, 287, - 288, 305, 306, 307, 308, 319, 326, 335, 349, 368, - 385, 402, 405, 413, 416, 417, 423, 432, 433, 437, - 442, 448, 450, 458, 459, 461, 462, 533, 534, 946, - 950, 952, 532, 3, 30, 37, 42, 49, 55, 79, - 81, 87, 99, 128, 166, 168, 171, 172, 188, 203, - 216, 217, 219, 229, 231, 241, 261, 270, 291, 293, - 312, 352, 369, 380, 398, 407, 426, 450, 452, 501, - 784, 829, 830, 872, 878, 946, 951, 501, 500, 613, - 361, 624, 267, 884, 40, 454, 186, 527, 186, 527, - 945, 186, 527, 186, 527, 87, 889, 148, 466, 88, - 125, 299, 410, 451, 771, 771, 771, 500, 759, 759, - 315, 500, 761, 148, 500, 69, 70, 771, 759, 756, - 464, 486, 500, 774, 500, 774, 500, 63, 348, 504, - 630, 500, 39, 755, 500, 107, 108, 183, 184, 247, - 248, 249, 250, 251, 252, 255, 256, 365, 366, 480, - 481, 500, 775, 776, 777, 778, 779, 780, 781, 782, - 783, 759, 148, 502, 876, 504, 630, 148, 504, 630, - 148, 282, 565, 566, 784, 878, 501, 504, 4, 155, - 282, 423, 491, 492, 530, 571, 574, 871, 900, 902, - 903, 906, 901, 416, 500, 644, 648, 167, 784, 827, - 500, 3, 530, 775, 776, 777, 778, 779, 780, 781, - 782, 832, 833, 736, 737, 784, 527, 784, 834, 491, - 492, 528, 785, 786, 809, 820, 836, 500, 784, 826, - 837, 784, 58, 167, 227, 417, 784, 827, 840, 784, - 501, 529, 500, 407, 692, 693, 693, 675, 676, 724, - 214, 670, 219, 37, 219, 369, 738, 219, 291, 739, - 724, 739, 219, 738, 500, 219, 739, 219, 144, 194, - 726, 219, 693, 500, 529, 500, 693, 289, 528, 530, - 917, 571, 906, 912, 914, 829, 754, 831, 40, 230, - 528, 500, 498, 655, 784, 846, 500, 655, 508, 784, - 500, 500, 784, 784, 784, 144, 858, 859, 784, 827, - 828, 655, 784, 826, 9, 534, 529, 810, 811, 812, - 502, 531, 849, 531, 500, 530, 500, 530, 871, 3, - 8, 11, 15, 16, 17, 18, 19, 20, 21, 36, - 40, 46, 53, 78, 172, 188, 193, 216, 217, 231, - 267, 270, 284, 287, 380, 484, 487, 488, 489, 491, - 492, 493, 494, 495, 496, 818, 819, 820, 822, 852, - 463, 795, 293, 784, 504, 670, 500, 530, 670, 3, - 113, 235, 530, 571, 773, 913, 102, 113, 914, 113, - 914, 528, 40, 528, 501, 504, 899, 504, 501, 668, - 865, 866, 40, 924, 187, 343, 214, 375, 657, 657, - 30, 680, 681, 784, 59, 657, 674, 159, 264, 700, - 221, 265, 329, 378, 439, 4, 9, 30, 695, 784, - 491, 492, 696, 697, 784, 786, 712, 713, 687, 686, - 684, 685, 162, 715, 277, 717, 59, 663, 664, 665, - 727, 785, 856, 856, 684, 712, 827, 877, 230, 527, - 72, 80, 91, 164, 186, 319, 433, 528, 595, 605, - 620, 80, 91, 536, 91, 536, 500, 416, 500, 593, - 239, 436, 593, 91, 504, 416, 527, 756, 571, 59, - 573, 571, 571, 104, 245, 253, 59, 416, 462, 486, - 570, 258, 361, 570, 572, 741, 91, 416, 536, 361, - 527, 416, 361, 528, 644, 499, 510, 829, 829, 830, - 504, 678, 679, 13, 14, 416, 528, 612, 617, 462, - 647, 527, 333, 370, 405, 454, 634, 148, 98, 562, - 578, 885, 886, 941, 140, 754, 527, 267, 563, 569, - 267, 500, 613, 40, 267, 613, 267, 500, 635, 186, - 528, 607, 890, 3, 500, 828, 871, 632, 831, 771, - 771, 39, 755, 414, 414, 871, 871, 528, 751, 754, - 751, 498, 498, 871, 871, 416, 416, 416, 416, 629, - 533, 502, 875, 865, 868, 868, 875, 501, 504, 13, - 901, 907, 4, 871, 4, 871, 530, 534, 646, 653, - 55, 100, 119, 137, 141, 163, 166, 181, 272, 280, - 301, 327, 650, 917, 40, 501, 784, 501, 167, 504, - 501, 308, 835, 501, 785, 785, 11, 15, 16, 19, - 20, 21, 193, 216, 284, 487, 488, 489, 491, 492, - 493, 494, 495, 496, 820, 785, 501, 743, 744, 792, - 162, 167, 838, 839, 504, 501, 40, 840, 827, 840, - 840, 167, 501, 40, 735, 500, 866, 4, 9, 528, - 688, 690, 691, 856, 854, 173, 235, 405, 409, 411, - 436, 527, 671, 459, 747, 724, 724, 219, 724, 282, - 442, 740, 724, 219, 856, 724, 724, 274, 274, 500, - 724, 529, 748, 749, 500, 529, 748, 504, 501, 504, - 501, 504, 537, 636, 637, 654, 882, 923, 827, 828, - 458, 843, 844, 784, 784, 813, 814, 815, 816, 8, - 15, 16, 19, 20, 21, 487, 488, 489, 491, 492, - 493, 494, 495, 496, 528, 818, 823, 501, 827, 500, - 528, 343, 863, 162, 499, 501, 504, 510, 504, 509, - 493, 532, 827, 784, 783, 783, 754, 784, 784, 784, - 784, 784, 784, 784, 784, 5, 534, 879, 414, 45, - 402, 853, 875, 784, 784, 500, 655, 841, 128, 155, - 267, 272, 277, 423, 434, 784, 272, 500, 784, 416, - 53, 172, 188, 193, 231, 380, 784, 784, 784, 784, - 784, 784, 784, 784, 784, 784, 30, 38, 385, 817, - 498, 502, 851, 176, 158, 796, 356, 500, 809, 857, - 167, 721, 829, 721, 500, 530, 528, 527, 919, 527, - 927, 784, 504, 501, 244, 267, 669, 442, 926, 527, - 539, 500, 528, 544, 554, 555, 557, 41, 122, 682, - 504, 442, 682, 258, 657, 356, 357, 491, 492, 697, - 699, 786, 378, 221, 283, 304, 304, 504, 495, 4, - 698, 871, 698, 356, 357, 699, 527, 864, 271, 382, - 718, 500, 866, 867, 504, 176, 442, 193, 176, 214, - 713, 685, 501, 341, 523, 500, 186, 605, 868, 221, - 267, 221, 442, 500, 598, 753, 754, 868, 528, 186, - 868, 186, 528, 26, 133, 375, 519, 522, 534, 589, - 603, 868, 534, 597, 616, 868, 520, 868, 341, 375, - 523, 569, 571, 875, 868, 571, 875, 868, 571, 341, - 375, 523, 868, 868, 868, 868, 341, 375, 523, 868, - 868, 492, 784, 848, 678, 678, 678, 450, 830, 187, - 346, 677, 784, 784, 324, 643, 501, 504, 280, 167, - 416, 638, 884, 454, 527, 527, 945, 527, 527, 527, - 287, 624, 500, 655, 500, 148, 148, 231, 528, 595, - 605, 608, 611, 621, 623, 462, 464, 600, 147, 654, - 148, 462, 891, 148, 501, 829, 40, 267, 282, 827, - 501, 501, 630, 501, 498, 483, 483, 501, 501, 754, - 501, 504, 501, 871, 499, 871, 501, 501, 776, 778, - 779, 780, 779, 780, 780, 630, 533, 630, 282, 630, - 565, 784, 644, 493, 500, 571, 645, 789, 906, 501, - 504, 40, 642, 530, 642, 267, 272, 327, 642, 59, - 642, 754, 501, 784, 784, 784, 838, 754, 785, 785, - 785, 785, 785, 785, 128, 267, 277, 785, 785, 785, - 785, 785, 785, 785, 785, 785, 785, 501, 504, 40, - 745, 784, 784, 839, 838, 754, 501, 501, 501, 827, - 754, 866, 501, 304, 495, 304, 357, 495, 500, 500, - 670, 409, 411, 409, 411, 527, 672, 672, 672, 784, - 176, 701, 740, 740, 724, 784, 500, 724, 162, 740, - 500, 529, 731, 740, 754, 501, 504, 748, 501, 912, - 754, 501, 499, 784, 135, 844, 845, 501, 510, 504, - 509, 502, 501, 501, 866, 500, 784, 860, 528, 784, - 784, 810, 859, 501, 501, 483, 785, 785, 141, 827, - 167, 128, 155, 272, 277, 423, 434, 500, 141, 823, - 784, 402, 853, 784, 841, 784, 416, 500, 655, 784, - 848, 533, 500, 500, 151, 797, 722, 723, 747, 678, - 747, 871, 783, 877, 877, 244, 500, 723, 459, 925, - 40, 59, 540, 550, 557, 849, 504, 721, 489, 485, - 683, 681, 284, 818, 821, 683, 4, 871, 699, 283, - 439, 696, 504, 238, 866, 663, 59, 856, 500, 529, - 59, 258, 416, 784, 267, 620, 500, 148, 500, 598, - 197, 617, 618, 579, 40, 171, 588, 614, 579, 26, - 133, 345, 347, 375, 516, 517, 518, 524, 525, 148, - 630, 148, 630, 589, 603, 589, 501, 504, 530, 582, - 489, 502, 501, 504, 416, 361, 91, 416, 536, 361, - 416, 416, 416, 361, 510, 499, 510, 677, 677, 677, - 830, 274, 274, 501, 392, 393, 530, 652, 612, 643, - 527, 563, 500, 40, 613, 635, 884, 343, 405, 530, - 559, 560, 617, 527, 527, 945, 527, 501, 504, 280, - 593, 280, 282, 592, 868, 462, 944, 527, 593, 40, - 527, 501, 405, 784, 148, 527, 501, 755, 871, 774, - 774, 755, 528, 755, 499, 499, 875, 640, 651, 906, - 646, 530, 530, 272, 617, 493, 617, 530, 493, 617, - 530, 501, 501, 839, 167, 128, 277, 500, 746, 743, - 500, 501, 501, 501, 528, 688, 747, 672, 672, 672, - 672, 527, 527, 527, 59, 180, 710, 740, 866, 500, - 728, 729, 730, 787, 869, 866, 162, 78, 750, 749, - 501, 413, 784, 139, 784, 813, 823, 501, 784, 860, - 861, 862, 40, 193, 501, 863, 783, 784, 36, 36, - 784, 501, 784, 167, 500, 831, 784, 501, 141, 785, - 785, 141, 141, 784, 784, 499, 510, 500, 850, 679, - 459, 784, 292, 801, 504, 701, 677, 701, 501, 882, - 784, 349, 548, 528, 258, 311, 113, 295, 500, 538, - 654, 501, 504, 544, 925, 784, 159, 225, 500, 683, - 283, 527, 501, 867, 176, 655, 656, 856, 867, 868, - 868, 501, 148, 618, 605, 618, 579, 607, 504, 501, - 115, 201, 265, 267, 604, 500, 33, 59, 625, 614, - 72, 78, 91, 113, 115, 201, 267, 272, 319, 335, - 433, 442, 584, 585, 599, 171, 113, 185, 267, 593, - 570, 105, 113, 171, 267, 391, 394, 572, 593, 375, - 518, 427, 868, 528, 522, 3, 37, 42, 49, 55, - 79, 81, 87, 99, 166, 168, 171, 172, 188, 203, - 216, 217, 219, 229, 231, 241, 261, 266, 270, 284, - 291, 293, 312, 352, 369, 376, 380, 398, 407, 426, - 432, 452, 491, 492, 530, 571, 580, 619, 754, 821, - 872, 946, 952, 534, 616, 868, 868, 868, 868, 868, - 868, 868, 868, 868, 868, 848, 848, 501, 501, 501, - 678, 570, 652, 500, 611, 654, 891, 40, 624, 186, - 527, 501, 504, 501, 563, 500, 40, 602, 600, 608, - 84, 567, 105, 265, 613, 654, 635, 654, 607, 442, - 888, 499, 754, 630, 501, 504, 617, 785, 167, 500, - 831, 748, 501, 504, 501, 701, 527, 527, 527, 527, - 30, 101, 177, 355, 500, 702, 703, 704, 705, 706, - 707, 708, 784, 784, 461, 798, 501, 786, 824, 825, - 193, 176, 725, 729, 500, 501, 731, 732, 733, 875, - 784, 504, 501, 528, 784, 786, 784, 784, 784, 831, - 501, 784, 36, 36, 784, 784, 141, 501, 492, 848, - 501, 829, 501, 784, 501, 500, 528, 802, 710, 501, - 710, 530, 501, 855, 448, 404, 441, 549, 528, 543, - 553, 282, 546, 489, 557, 548, 823, 59, 501, 501, - 447, 448, 660, 579, 605, 501, 501, 462, 610, 116, - 189, 199, 115, 444, 784, 113, 40, 500, 875, 868, - 785, 116, 189, 115, 272, 221, 527, 610, 86, 625, - 186, 272, 571, 784, 625, 272, 491, 492, 574, 528, - 754, 630, 630, 241, 398, 872, 876, 489, 416, 416, - 499, 499, 677, 644, 442, 639, 641, 617, 501, 944, - 40, 405, 784, 405, 267, 500, 530, 891, 611, 147, - 654, 145, 195, 592, 118, 133, 318, 944, 105, 891, - 462, 942, 40, 282, 528, 887, 500, 651, 785, 831, - 501, 501, 9, 342, 694, 710, 500, 377, 500, 501, - 504, 528, 799, 800, 326, 711, 504, 501, 500, 529, - 59, 501, 829, 193, 501, 732, 860, 499, 186, 501, - 784, 784, 784, 510, 499, 510, 501, 501, 528, 803, - 798, 530, 798, 504, 447, 849, 501, 504, 89, 548, - 784, 501, 867, 867, 335, 610, 500, 601, 579, 501, - 185, 500, 784, 267, 585, 610, 613, 868, 40, 148, - 750, 876, 495, 580, 868, 868, 501, 570, 120, 501, - 600, 654, 654, 527, 148, 40, 501, 868, 944, 30, - 83, 92, 114, 185, 198, 391, 394, 596, 596, 357, - 357, 40, 64, 72, 235, 405, 784, 527, 500, 528, - 547, 556, 792, 501, 501, 500, 798, 827, 500, 827, - 704, 40, 504, 784, 442, 689, 786, 856, 866, 736, - 678, 500, 736, 784, 848, 848, 301, 804, 711, 711, - 654, 295, 654, 543, 282, 500, 541, 527, 579, 534, - 606, 609, 395, 456, 586, 587, 500, 581, 784, 501, - 243, 622, 185, 442, 521, 495, 427, 644, 530, 891, - 592, 942, 500, 527, 654, 600, 567, 654, 72, 285, - 72, 654, 888, 784, 78, 551, 501, 504, 551, 9, - 711, 501, 703, 501, 802, 800, 359, 501, 677, 856, - 499, 499, 499, 59, 678, 689, 689, 549, 91, 556, - 129, 613, 489, 501, 504, 569, 501, 265, 594, 168, - 300, 381, 282, 590, 591, 615, 581, 784, 427, 40, - 500, 942, 592, 944, 942, 285, 285, 500, 501, 875, - 552, 875, 891, 547, 552, 501, 689, 501, 691, 501, - 501, 826, 328, 357, 805, 447, 868, 501, 268, 439, - 622, 580, 609, 501, 587, 199, 118, 439, 282, 615, - 282, 590, 654, 556, 551, 682, 747, 682, 53, 102, - 429, 784, 806, 807, 806, 501, 654, 747, 375, 591, - 63, 265, 348, 375, 583, 583, 942, 501, 552, 683, - 683, 807, 356, 161, 314, 161, 314, 555, 579, 25, - 113, 272, 891, 682, 36, 747, 747, 683, 807 + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 531, 793, 873, 877, 950, 951, 952, 3, 171, + 242, 400, 531, 899, 950, 579, 55, 503, 652, 173, + 236, 288, 363, 411, 413, 430, 436, 439, 627, 896, + 5, 30, 318, 531, 532, 872, 106, 934, 24, 79, + 95, 142, 152, 164, 169, 198, 241, 245, 312, 327, + 360, 363, 372, 375, 393, 407, 414, 424, 430, 457, + 630, 631, 634, 579, 872, 455, 503, 518, 529, 538, + 540, 561, 564, 565, 567, 571, 578, 581, 629, 636, + 639, 640, 657, 884, 887, 890, 897, 898, 905, 909, + 913, 919, 922, 927, 936, 939, 944, 945, 947, 106, + 73, 209, 66, 77, 79, 154, 227, 274, 284, 296, + 313, 359, 402, 423, 425, 429, 451, 503, 530, 531, + 532, 658, 723, 725, 727, 737, 744, 745, 793, 795, + 796, 106, 5, 531, 533, 920, 920, 531, 872, 30, + 173, 236, 376, 417, 421, 531, 937, 938, 942, 579, + 30, 128, 678, 679, 173, 236, 363, 376, 417, 914, + 915, 942, 579, 531, 657, 669, 943, 531, 744, 407, + 675, 530, 168, 503, 924, 503, 335, 670, 671, 872, + 670, 658, 659, 0, 506, 118, 208, 442, 143, 213, + 289, 435, 681, 682, 727, 727, 658, 660, 662, 507, + 455, 903, 30, 417, 421, 657, 943, 187, 530, 872, + 187, 530, 187, 744, 187, 530, 533, 501, 505, 534, + 535, 503, 657, 530, 411, 413, 411, 413, 345, 187, + 530, 530, 169, 241, 335, 372, 407, 457, 637, 198, + 30, 872, 187, 3, 247, 424, 105, 407, 407, 457, + 366, 3, 37, 42, 49, 54, 55, 56, 57, 69, + 70, 79, 81, 87, 99, 110, 111, 132, 160, 166, + 168, 172, 187, 189, 204, 211, 212, 214, 217, 218, + 220, 230, 232, 242, 261, 262, 263, 271, 276, 293, + 295, 314, 332, 354, 358, 371, 378, 382, 385, 400, + 409, 417, 418, 429, 435, 452, 455, 635, 755, 757, + 759, 761, 763, 765, 767, 768, 769, 771, 772, 773, + 775, 776, 878, 950, 953, 187, 531, 632, 879, 187, + 869, 872, 187, 872, 503, 570, 905, 3, 37, 42, + 49, 53, 54, 55, 56, 57, 69, 70, 77, 79, + 81, 87, 99, 110, 111, 148, 154, 160, 166, 168, + 171, 172, 177, 178, 189, 204, 205, 211, 212, 214, + 217, 218, 220, 230, 232, 242, 261, 262, 263, 267, + 271, 274, 276, 292, 293, 295, 296, 313, 314, 317, + 332, 354, 358, 371, 378, 382, 385, 400, 402, 409, + 417, 418, 423, 425, 429, 451, 452, 455, 472, 473, + 474, 475, 476, 477, 478, 479, 480, 481, 482, 906, + 908, 909, 911, 912, 950, 954, 903, 531, 533, 899, + 920, 503, 503, 162, 503, 658, 745, 503, 503, 530, + 503, 503, 167, 503, 503, 503, 503, 658, 723, 727, + 737, 496, 534, 40, 531, 738, 739, 738, 370, 507, + 661, 37, 42, 99, 168, 204, 220, 230, 262, 307, + 314, 354, 371, 440, 741, 739, 40, 531, 738, 740, + 489, 749, 533, 167, 492, 503, 503, 885, 938, 938, + 938, 486, 219, 505, 283, 4, 6, 7, 8, 9, + 10, 39, 54, 56, 57, 65, 69, 70, 81, 110, + 111, 113, 148, 155, 160, 177, 178, 211, 212, 214, + 242, 261, 263, 268, 273, 276, 285, 332, 358, 385, + 417, 418, 426, 452, 487, 494, 495, 496, 501, 503, + 508, 509, 510, 511, 531, 533, 658, 712, 760, 763, + 766, 767, 768, 770, 771, 772, 775, 776, 787, 789, + 790, 791, 792, 793, 794, 796, 797, 812, 813, 824, + 846, 851, 859, 860, 861, 873, 874, 875, 858, 860, + 914, 914, 533, 914, 486, 167, 419, 492, 505, 534, + 744, 928, 3, 166, 168, 455, 909, 923, 925, 166, + 926, 787, 830, 831, 670, 507, 503, 881, 504, 504, + 517, 167, 288, 548, 928, 30, 128, 676, 676, 59, + 676, 157, 162, 233, 280, 687, 689, 690, 715, 717, + 718, 719, 176, 283, 445, 283, 681, 682, 503, 530, + 408, 941, 486, 219, 148, 26, 32, 133, 287, 343, + 347, 377, 448, 523, 526, 527, 343, 148, 40, 60, + 104, 197, 246, 254, 266, 298, 343, 349, 372, 377, + 391, 526, 572, 575, 148, 343, 377, 526, 148, 343, + 377, 526, 40, 935, 787, 852, 3, 30, 34, 35, + 36, 38, 39, 40, 41, 45, 58, 65, 66, 72, + 78, 80, 91, 98, 113, 115, 122, 128, 129, 135, + 139, 143, 155, 157, 162, 164, 167, 174, 176, 181, + 194, 202, 213, 215, 227, 228, 233, 268, 273, 280, + 283, 284, 288, 289, 307, 308, 309, 310, 321, 328, + 337, 351, 370, 387, 404, 407, 415, 419, 420, 426, + 435, 436, 440, 445, 451, 453, 461, 462, 464, 465, + 536, 537, 950, 954, 956, 535, 3, 30, 37, 42, + 49, 55, 79, 81, 87, 99, 128, 166, 168, 171, + 172, 189, 204, 217, 218, 220, 230, 232, 242, 262, + 271, 293, 295, 314, 354, 371, 382, 400, 409, 429, + 453, 455, 504, 787, 833, 834, 876, 882, 950, 955, + 504, 503, 616, 363, 627, 268, 888, 40, 457, 187, + 530, 187, 530, 949, 187, 530, 187, 530, 87, 893, + 148, 469, 88, 125, 301, 412, 454, 774, 774, 774, + 503, 762, 762, 317, 503, 764, 148, 503, 69, 70, + 774, 762, 759, 467, 489, 503, 777, 503, 777, 503, + 63, 350, 507, 633, 503, 39, 758, 503, 107, 108, + 184, 185, 248, 249, 250, 251, 252, 253, 256, 257, + 367, 368, 483, 484, 503, 778, 779, 780, 781, 782, + 783, 784, 785, 786, 762, 148, 505, 880, 507, 633, + 148, 507, 633, 148, 283, 568, 569, 787, 882, 504, + 507, 4, 155, 283, 426, 494, 495, 533, 574, 577, + 875, 904, 906, 907, 910, 905, 419, 503, 647, 651, + 167, 787, 831, 503, 3, 533, 778, 779, 780, 781, + 782, 783, 784, 785, 836, 837, 739, 740, 787, 530, + 787, 838, 494, 495, 531, 788, 789, 813, 824, 840, + 503, 787, 830, 841, 787, 58, 167, 228, 420, 787, + 831, 844, 787, 504, 532, 503, 409, 695, 696, 696, + 678, 679, 727, 215, 673, 220, 37, 220, 371, 741, + 220, 293, 742, 727, 742, 220, 741, 503, 220, 742, + 220, 144, 195, 729, 220, 696, 503, 532, 503, 696, + 290, 531, 533, 921, 574, 910, 916, 918, 833, 757, + 835, 40, 231, 531, 503, 501, 658, 787, 850, 503, + 658, 511, 787, 503, 503, 787, 787, 787, 144, 862, + 863, 787, 831, 832, 658, 787, 830, 9, 537, 532, + 814, 815, 816, 505, 534, 853, 534, 503, 533, 503, + 533, 875, 3, 8, 11, 15, 16, 17, 18, 19, + 20, 21, 36, 40, 46, 53, 78, 172, 189, 194, + 217, 218, 232, 268, 271, 285, 288, 382, 487, 490, + 491, 492, 494, 495, 496, 497, 498, 499, 822, 823, + 824, 826, 856, 466, 798, 295, 787, 507, 673, 503, + 533, 673, 3, 113, 236, 533, 574, 776, 917, 102, + 113, 918, 113, 918, 531, 40, 531, 504, 507, 903, + 507, 504, 671, 869, 870, 40, 928, 188, 345, 215, + 377, 660, 660, 30, 683, 684, 787, 59, 660, 677, + 159, 265, 703, 222, 266, 331, 380, 442, 4, 9, + 30, 698, 787, 494, 495, 699, 700, 787, 789, 715, + 716, 690, 689, 687, 688, 162, 718, 278, 720, 59, + 666, 667, 668, 730, 788, 860, 860, 687, 715, 831, + 881, 231, 530, 72, 80, 91, 164, 187, 321, 436, + 531, 598, 608, 623, 80, 91, 539, 91, 539, 503, + 419, 503, 596, 240, 439, 596, 91, 507, 419, 530, + 759, 574, 59, 576, 574, 574, 104, 246, 254, 59, + 419, 465, 489, 573, 259, 363, 573, 575, 744, 91, + 419, 539, 363, 530, 419, 363, 531, 647, 502, 513, + 833, 833, 834, 507, 681, 682, 13, 14, 419, 531, + 615, 620, 465, 650, 530, 335, 372, 407, 457, 637, + 148, 98, 565, 581, 889, 890, 945, 140, 757, 530, + 268, 566, 572, 268, 503, 616, 40, 268, 616, 268, + 503, 638, 187, 531, 610, 894, 3, 503, 832, 875, + 635, 835, 774, 774, 39, 758, 417, 417, 875, 875, + 531, 754, 757, 754, 501, 501, 875, 875, 419, 419, + 419, 419, 632, 536, 505, 879, 869, 872, 872, 879, + 504, 507, 13, 905, 911, 4, 875, 4, 875, 533, + 537, 649, 656, 55, 100, 119, 137, 141, 163, 166, + 182, 273, 281, 303, 329, 653, 921, 40, 504, 787, + 504, 167, 507, 504, 310, 839, 504, 788, 788, 11, + 15, 16, 19, 20, 21, 194, 217, 285, 490, 491, + 492, 494, 495, 496, 497, 498, 499, 824, 788, 504, + 746, 747, 795, 162, 167, 842, 843, 507, 504, 40, + 844, 831, 844, 844, 167, 504, 40, 738, 503, 870, + 4, 9, 531, 691, 693, 694, 860, 858, 173, 236, + 407, 411, 413, 439, 530, 674, 462, 750, 727, 727, + 220, 727, 283, 445, 743, 727, 220, 860, 727, 727, + 275, 275, 503, 727, 532, 751, 752, 503, 532, 751, + 507, 504, 507, 504, 507, 540, 639, 640, 657, 886, + 927, 831, 832, 461, 847, 848, 787, 787, 817, 818, + 819, 820, 8, 15, 16, 19, 20, 21, 490, 491, + 492, 494, 495, 496, 497, 498, 499, 531, 822, 827, + 504, 831, 503, 531, 345, 867, 162, 502, 504, 507, + 513, 507, 512, 496, 535, 831, 787, 786, 786, 757, + 787, 787, 787, 787, 787, 787, 787, 787, 5, 537, + 883, 417, 45, 404, 857, 879, 787, 787, 503, 658, + 845, 128, 155, 268, 273, 278, 426, 437, 787, 273, + 503, 787, 419, 53, 172, 189, 194, 232, 382, 787, + 787, 787, 787, 787, 787, 787, 787, 787, 787, 30, + 38, 387, 821, 501, 505, 855, 176, 158, 799, 358, + 503, 813, 861, 167, 724, 833, 724, 503, 533, 531, + 530, 923, 530, 931, 787, 507, 504, 245, 268, 672, + 445, 930, 530, 542, 503, 531, 547, 557, 558, 560, + 41, 122, 685, 507, 445, 685, 259, 660, 358, 359, + 494, 495, 700, 702, 789, 380, 222, 284, 306, 306, + 507, 498, 4, 701, 875, 701, 358, 359, 702, 530, + 868, 272, 384, 721, 503, 870, 871, 507, 176, 445, + 194, 176, 215, 716, 688, 504, 343, 526, 503, 187, + 608, 872, 222, 268, 222, 445, 503, 601, 756, 757, + 872, 531, 187, 872, 187, 531, 26, 133, 377, 522, + 525, 537, 592, 606, 872, 537, 600, 619, 872, 523, + 872, 343, 377, 526, 572, 574, 879, 872, 574, 879, + 872, 574, 343, 377, 526, 872, 872, 872, 872, 343, + 377, 526, 872, 872, 495, 787, 852, 681, 681, 681, + 453, 834, 188, 348, 680, 787, 787, 326, 646, 504, + 507, 281, 167, 419, 641, 888, 457, 530, 530, 949, + 530, 530, 530, 288, 627, 503, 658, 503, 148, 148, + 232, 531, 598, 608, 611, 614, 624, 626, 465, 467, + 603, 147, 657, 148, 465, 895, 148, 504, 833, 40, + 268, 283, 831, 504, 504, 633, 504, 501, 486, 486, + 504, 504, 757, 504, 507, 504, 875, 502, 875, 504, + 504, 779, 781, 782, 783, 782, 783, 783, 633, 536, + 633, 283, 633, 568, 787, 647, 496, 503, 574, 648, + 792, 910, 504, 507, 40, 645, 533, 645, 268, 273, + 329, 645, 59, 645, 757, 504, 787, 787, 787, 842, + 757, 788, 788, 788, 788, 788, 788, 128, 268, 278, + 788, 788, 788, 788, 788, 788, 788, 788, 788, 788, + 504, 507, 40, 748, 787, 787, 843, 842, 757, 504, + 504, 504, 831, 757, 870, 504, 306, 498, 306, 359, + 498, 503, 503, 673, 411, 413, 411, 413, 530, 675, + 675, 675, 787, 176, 704, 743, 743, 727, 787, 503, + 727, 162, 743, 503, 532, 734, 743, 757, 504, 507, + 751, 504, 916, 757, 504, 502, 787, 135, 848, 849, + 504, 513, 507, 512, 505, 504, 504, 870, 503, 787, + 864, 531, 787, 787, 814, 863, 504, 504, 486, 788, + 788, 141, 831, 167, 128, 155, 273, 278, 426, 437, + 503, 141, 827, 787, 404, 857, 787, 845, 787, 419, + 503, 658, 787, 852, 536, 503, 503, 151, 800, 725, + 726, 750, 681, 750, 875, 786, 881, 881, 245, 503, + 726, 462, 929, 40, 59, 543, 553, 560, 853, 507, + 724, 492, 488, 686, 684, 285, 822, 825, 686, 4, + 875, 702, 284, 442, 699, 507, 239, 870, 666, 59, + 860, 503, 532, 59, 259, 419, 787, 268, 623, 503, + 148, 503, 601, 198, 620, 621, 582, 40, 171, 591, + 617, 582, 26, 133, 347, 349, 377, 519, 520, 521, + 527, 528, 148, 633, 148, 633, 592, 606, 592, 504, + 507, 533, 585, 492, 505, 504, 507, 419, 363, 91, + 419, 539, 363, 419, 419, 419, 363, 513, 502, 513, + 680, 680, 680, 834, 275, 275, 504, 394, 395, 533, + 655, 615, 646, 530, 566, 503, 40, 616, 638, 888, + 345, 407, 533, 562, 563, 620, 530, 530, 949, 530, + 504, 507, 281, 596, 281, 283, 595, 872, 465, 948, + 530, 596, 40, 530, 504, 407, 787, 148, 530, 504, + 758, 875, 777, 777, 758, 531, 758, 502, 502, 879, + 643, 654, 910, 649, 533, 533, 273, 620, 496, 620, + 533, 496, 620, 533, 504, 504, 843, 167, 128, 278, + 503, 749, 746, 503, 504, 504, 504, 531, 691, 750, + 675, 675, 675, 675, 530, 530, 530, 59, 181, 713, + 743, 870, 503, 731, 732, 733, 790, 873, 870, 162, + 78, 753, 752, 504, 415, 787, 139, 787, 817, 827, + 504, 787, 864, 865, 866, 40, 194, 504, 867, 786, + 787, 36, 36, 787, 504, 787, 167, 503, 835, 787, + 504, 141, 788, 788, 141, 141, 787, 787, 502, 513, + 503, 854, 682, 462, 787, 294, 804, 507, 704, 680, + 704, 504, 886, 787, 351, 551, 531, 259, 313, 113, + 297, 503, 541, 657, 504, 507, 547, 929, 787, 159, + 226, 503, 686, 284, 530, 504, 871, 176, 658, 659, + 860, 871, 872, 872, 504, 148, 621, 608, 621, 582, + 610, 507, 504, 115, 202, 266, 268, 607, 503, 33, + 59, 628, 617, 72, 78, 91, 113, 115, 202, 268, + 273, 321, 337, 436, 445, 587, 588, 602, 171, 113, + 186, 268, 596, 573, 105, 113, 171, 268, 393, 396, + 575, 596, 377, 521, 430, 872, 531, 525, 3, 37, + 42, 49, 55, 79, 81, 87, 99, 166, 168, 171, + 172, 189, 204, 217, 218, 220, 230, 232, 242, 262, + 267, 271, 285, 293, 295, 314, 354, 371, 378, 382, + 400, 409, 429, 435, 455, 494, 495, 533, 574, 583, + 622, 757, 825, 876, 950, 956, 537, 619, 872, 872, + 872, 872, 872, 872, 872, 872, 872, 872, 852, 852, + 504, 504, 504, 681, 573, 655, 503, 614, 657, 895, + 40, 627, 187, 530, 504, 507, 504, 566, 503, 40, + 605, 603, 611, 84, 570, 105, 266, 616, 657, 638, + 657, 610, 445, 892, 502, 757, 633, 504, 507, 620, + 788, 167, 503, 835, 751, 504, 507, 504, 704, 530, + 530, 530, 530, 30, 101, 177, 357, 503, 705, 706, + 707, 708, 709, 710, 711, 787, 787, 464, 801, 504, + 789, 828, 829, 194, 176, 728, 732, 503, 504, 734, + 735, 736, 879, 787, 507, 504, 531, 787, 789, 787, + 787, 787, 835, 504, 787, 36, 36, 787, 787, 141, + 504, 495, 852, 504, 833, 504, 787, 504, 503, 531, + 805, 713, 504, 713, 533, 504, 859, 451, 406, 444, + 552, 531, 546, 556, 283, 549, 492, 560, 551, 827, + 59, 504, 504, 450, 451, 663, 582, 608, 504, 504, + 465, 613, 116, 190, 200, 115, 447, 787, 113, 40, + 503, 879, 872, 788, 116, 190, 115, 273, 222, 530, + 613, 86, 628, 187, 273, 574, 787, 628, 273, 494, + 495, 577, 531, 757, 633, 633, 242, 400, 876, 880, + 492, 419, 419, 502, 502, 680, 647, 445, 642, 644, + 620, 504, 948, 40, 407, 787, 407, 268, 503, 533, + 895, 614, 147, 657, 145, 196, 595, 118, 133, 320, + 948, 105, 895, 465, 946, 40, 283, 531, 891, 503, + 654, 788, 835, 504, 504, 9, 344, 697, 713, 503, + 379, 503, 504, 507, 531, 802, 803, 328, 714, 507, + 504, 503, 532, 59, 504, 833, 194, 504, 735, 864, + 502, 187, 504, 787, 787, 787, 513, 502, 513, 504, + 504, 531, 806, 801, 533, 801, 507, 450, 853, 504, + 507, 89, 551, 787, 504, 871, 871, 337, 613, 503, + 604, 582, 504, 186, 503, 787, 268, 588, 613, 616, + 872, 40, 148, 753, 880, 498, 583, 872, 872, 504, + 573, 120, 504, 603, 657, 657, 530, 148, 40, 504, + 872, 948, 30, 83, 92, 114, 186, 199, 393, 396, + 599, 599, 359, 359, 40, 64, 72, 236, 407, 787, + 530, 503, 531, 550, 559, 795, 504, 504, 503, 801, + 831, 503, 831, 707, 40, 507, 787, 445, 692, 789, + 860, 870, 739, 681, 503, 739, 787, 852, 852, 303, + 807, 714, 714, 657, 297, 657, 546, 283, 503, 544, + 530, 582, 537, 609, 612, 397, 459, 589, 590, 503, + 584, 787, 504, 244, 625, 186, 445, 524, 498, 430, + 647, 533, 895, 595, 946, 503, 530, 657, 603, 570, + 657, 72, 286, 72, 657, 892, 787, 78, 554, 504, + 507, 554, 9, 714, 504, 706, 504, 805, 803, 361, + 504, 680, 860, 502, 502, 502, 59, 681, 692, 692, + 552, 91, 559, 129, 616, 492, 504, 507, 572, 504, + 266, 597, 168, 302, 383, 283, 593, 594, 618, 584, + 787, 430, 40, 503, 946, 595, 948, 946, 286, 286, + 503, 504, 879, 555, 879, 895, 550, 555, 504, 692, + 504, 694, 504, 504, 830, 179, 330, 359, 808, 450, + 872, 504, 269, 442, 625, 583, 612, 504, 590, 200, + 118, 442, 283, 618, 283, 593, 657, 559, 554, 685, + 750, 685, 53, 102, 432, 787, 809, 810, 809, 809, + 504, 657, 750, 377, 594, 63, 266, 350, 377, 586, + 586, 946, 504, 555, 686, 686, 810, 358, 161, 316, + 161, 316, 144, 811, 811, 811, 558, 582, 25, 113, + 273, 895, 685, 36, 102, 176, 266, 416, 750, 750, + 686, 810, 358, 291 }; #define yyerrok (yyerrstatus = 0) @@ -19122,14 +19246,14 @@ YYLTYPE yylloc; switch (yyn) { case 2: -#line 493 "third_party/libpg_query/grammar/grammar.y" +#line 494 "third_party/libpg_query/grammar/grammar.y" { pg_yyget_extra(yyscanner)->parsetree = (yyvsp[(1) - (1)].list); ;} break; case 3: -#line 509 "third_party/libpg_query/grammar/grammar.y" +#line 510 "third_party/libpg_query/grammar/grammar.y" { if ((yyvsp[(1) - (3)].list) != NIL) { @@ -19144,7 +19268,7 @@ YYLTYPE yylloc; break; case 4: -#line 521 "third_party/libpg_query/grammar/grammar.y" +#line 522 "third_party/libpg_query/grammar/grammar.y" { if ((yyvsp[(1) - (1)].node) != NULL) (yyval.list) = list_make1(makeRawStmt((yyvsp[(1) - (1)].node), 0)); @@ -19154,7 +19278,7 @@ YYLTYPE yylloc; break; case 42: -#line 567 "third_party/libpg_query/grammar/grammar.y" +#line 568 "third_party/libpg_query/grammar/grammar.y" { (yyval.node) = NULL; ;} break; @@ -26713,27 +26837,43 @@ YYLTYPE yylloc; break; case 1028: -#line 3109 "third_party/libpg_query/grammar/statements/select.y" +#line 3106 "third_party/libpg_query/grammar/statements/select.y" { - PGWindowDef *n = (yyvsp[(2) - (2)].windef); + PGWindowDef *n = (yyvsp[(2) - (3)].windef); + n->frameOptions |= FRAMEOPTION_NONDEFAULT | FRAMEOPTION_RANGE; + n->frameOptions |= (yyvsp[(3) - (3)].ival); (yyval.windef) = n; ;} break; case 1029: -#line 3115 "third_party/libpg_query/grammar/statements/select.y" +#line 3114 "third_party/libpg_query/grammar/statements/select.y" { - PGWindowDef *n = (yyvsp[(2) - (2)].windef); + PGWindowDef *n = (yyvsp[(2) - (3)].windef); + n->frameOptions |= FRAMEOPTION_NONDEFAULT | FRAMEOPTION_ROWS; + n->frameOptions |= (yyvsp[(3) - (3)].ival); (yyval.windef) = n; ;} break; case 1030: -#line 3121 "third_party/libpg_query/grammar/statements/select.y" +#line 3122 "third_party/libpg_query/grammar/statements/select.y" + { + PGWindowDef *n = (yyvsp[(2) - (3)].windef); + + n->frameOptions |= FRAMEOPTION_NONDEFAULT | FRAMEOPTION_GROUPS; + n->frameOptions |= (yyvsp[(3) - (3)].ival); + (yyval.windef) = n; + ;} + break; + + case 1031: +#line 3130 "third_party/libpg_query/grammar/statements/select.y" { PGWindowDef *n = makeNode(PGWindowDef); + n->frameOptions = FRAMEOPTION_DEFAULTS; n->startOffset = NULL; n->endOffset = NULL; @@ -26741,17 +26881,18 @@ YYLTYPE yylloc; ;} break; - case 1031: -#line 3131 "third_party/libpg_query/grammar/statements/select.y" + case 1032: +#line 3141 "third_party/libpg_query/grammar/statements/select.y" { PGWindowDef *n = (yyvsp[(1) - (1)].windef); + /* reject invalid cases */ if (n->frameOptions & FRAMEOPTION_START_UNBOUNDED_FOLLOWING) ereport(ERROR, (errcode(PG_ERRCODE_WINDOWING_ERROR), errmsg("frame start cannot be UNBOUNDED FOLLOWING"), parser_errposition((yylsp[(1) - (1)])))); - if (n->frameOptions & FRAMEOPTION_START_VALUE_FOLLOWING) + if (n->frameOptions & FRAMEOPTION_START_OFFSET_FOLLOWING) ereport(ERROR, (errcode(PG_ERRCODE_WINDOWING_ERROR), errmsg("frame starting from following row cannot end with current row"), @@ -26761,11 +26902,12 @@ YYLTYPE yylloc; ;} break; - case 1032: -#line 3148 "third_party/libpg_query/grammar/statements/select.y" + case 1033: +#line 3159 "third_party/libpg_query/grammar/statements/select.y" { PGWindowDef *n1 = (yyvsp[(2) - (4)].windef); PGWindowDef *n2 = (yyvsp[(4) - (4)].windef); + /* form merged options */ int frameOptions = n1->frameOptions; /* shift converts START_ options to END_ options */ @@ -26783,13 +26925,13 @@ YYLTYPE yylloc; errmsg("frame end cannot be UNBOUNDED PRECEDING"), parser_errposition((yylsp[(4) - (4)])))); if ((frameOptions & FRAMEOPTION_START_CURRENT_ROW) && - (frameOptions & FRAMEOPTION_END_VALUE_PRECEDING)) + (frameOptions & FRAMEOPTION_END_OFFSET_PRECEDING)) ereport(ERROR, (errcode(PG_ERRCODE_WINDOWING_ERROR), errmsg("frame starting from current row cannot have preceding rows"), parser_errposition((yylsp[(4) - (4)])))); - if ((frameOptions & FRAMEOPTION_START_VALUE_FOLLOWING) && - (frameOptions & (FRAMEOPTION_END_VALUE_PRECEDING | + if ((frameOptions & FRAMEOPTION_START_OFFSET_FOLLOWING) && + (frameOptions & (FRAMEOPTION_END_OFFSET_PRECEDING | FRAMEOPTION_END_CURRENT_ROW))) ereport(ERROR, (errcode(PG_ERRCODE_WINDOWING_ERROR), @@ -26801,10 +26943,11 @@ YYLTYPE yylloc; ;} break; - case 1033: -#line 3193 "third_party/libpg_query/grammar/statements/select.y" + case 1034: +#line 3205 "third_party/libpg_query/grammar/statements/select.y" { PGWindowDef *n = makeNode(PGWindowDef); + n->frameOptions = FRAMEOPTION_START_UNBOUNDED_PRECEDING; n->startOffset = NULL; n->endOffset = NULL; @@ -26812,10 +26955,11 @@ YYLTYPE yylloc; ;} break; - case 1034: -#line 3201 "third_party/libpg_query/grammar/statements/select.y" + case 1035: +#line 3214 "third_party/libpg_query/grammar/statements/select.y" { PGWindowDef *n = makeNode(PGWindowDef); + n->frameOptions = FRAMEOPTION_START_UNBOUNDED_FOLLOWING; n->startOffset = NULL; n->endOffset = NULL; @@ -26823,10 +26967,11 @@ YYLTYPE yylloc; ;} break; - case 1035: -#line 3209 "third_party/libpg_query/grammar/statements/select.y" + case 1036: +#line 3223 "third_party/libpg_query/grammar/statements/select.y" { PGWindowDef *n = makeNode(PGWindowDef); + n->frameOptions = FRAMEOPTION_START_CURRENT_ROW; n->startOffset = NULL; n->endOffset = NULL; @@ -26834,50 +26979,77 @@ YYLTYPE yylloc; ;} break; - case 1036: -#line 3217 "third_party/libpg_query/grammar/statements/select.y" + case 1037: +#line 3232 "third_party/libpg_query/grammar/statements/select.y" { PGWindowDef *n = makeNode(PGWindowDef); - n->frameOptions = FRAMEOPTION_START_VALUE_PRECEDING; + + n->frameOptions = FRAMEOPTION_START_OFFSET_PRECEDING; n->startOffset = (yyvsp[(1) - (2)].node); n->endOffset = NULL; (yyval.windef) = n; ;} break; - case 1037: -#line 3225 "third_party/libpg_query/grammar/statements/select.y" + case 1038: +#line 3241 "third_party/libpg_query/grammar/statements/select.y" { PGWindowDef *n = makeNode(PGWindowDef); - n->frameOptions = FRAMEOPTION_START_VALUE_FOLLOWING; + + n->frameOptions = FRAMEOPTION_START_OFFSET_FOLLOWING; n->startOffset = (yyvsp[(1) - (2)].node); n->endOffset = NULL; (yyval.windef) = n; ;} break; - case 1038: -#line 3245 "third_party/libpg_query/grammar/statements/select.y" + case 1039: +#line 3252 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = FRAMEOPTION_EXCLUDE_CURRENT_ROW; ;} + break; + + case 1040: +#line 3253 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = FRAMEOPTION_EXCLUDE_GROUP; ;} + break; + + case 1041: +#line 3254 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = FRAMEOPTION_EXCLUDE_TIES; ;} + break; + + case 1042: +#line 3255 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = 0; ;} + break; + + case 1043: +#line 3256 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = 0; ;} + break; + + case 1044: +#line 3270 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(3) - (4)].list); ;} break; - case 1039: -#line 3246 "third_party/libpg_query/grammar/statements/select.y" + case 1045: +#line 3271 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NIL; ;} break; - case 1040: -#line 3249 "third_party/libpg_query/grammar/statements/select.y" + case 1046: +#line 3274 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list);;} break; - case 1041: -#line 3250 "third_party/libpg_query/grammar/statements/select.y" + case 1047: +#line 3275 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(2) - (5)].list), (yyvsp[(4) - (5)].node)); ;} break; - case 1042: -#line 3254 "third_party/libpg_query/grammar/statements/select.y" + case 1048: +#line 3279 "third_party/libpg_query/grammar/statements/select.y" { PGNamedArgExpr *na = makeNode(PGNamedArgExpr); na->name = (yyvsp[(1) - (3)].str); @@ -26888,321 +27060,321 @@ YYLTYPE yylloc; ;} break; - case 1043: -#line 3264 "third_party/libpg_query/grammar/statements/select.y" + case 1049: +#line 3289 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} break; - case 1044: -#line 3265 "third_party/libpg_query/grammar/statements/select.y" + case 1050: +#line 3290 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} break; - case 1045: -#line 3269 "third_party/libpg_query/grammar/statements/select.y" + case 1051: +#line 3294 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1046: -#line 3270 "third_party/libpg_query/grammar/statements/select.y" + case 1052: +#line 3295 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (2)].list); ;} break; - case 1047: -#line 3275 "third_party/libpg_query/grammar/statements/select.y" + case 1053: +#line 3300 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make2((yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node)); ;} break; - case 1048: -#line 3281 "third_party/libpg_query/grammar/statements/select.y" + case 1054: +#line 3306 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].list)); ;} break; - case 1049: -#line 3282 "third_party/libpg_query/grammar/statements/select.y" + case 1055: +#line 3307 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].list)); ;} break; - case 1050: -#line 3287 "third_party/libpg_query/grammar/statements/select.y" + case 1056: +#line 3312 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1051: -#line 3288 "third_party/libpg_query/grammar/statements/select.y" + case 1057: +#line 3313 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (2)].list); ;} break; - case 1052: -#line 3293 "third_party/libpg_query/grammar/statements/select.y" + case 1058: +#line 3318 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1053: -#line 3294 "third_party/libpg_query/grammar/statements/select.y" + case 1059: +#line 3319 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NULL; ;} break; - case 1054: -#line 3297 "third_party/libpg_query/grammar/statements/select.y" + case 1060: +#line 3322 "third_party/libpg_query/grammar/statements/select.y" { (yyval.subquerytype) = PG_ANY_SUBLINK; ;} break; - case 1055: -#line 3298 "third_party/libpg_query/grammar/statements/select.y" + case 1061: +#line 3323 "third_party/libpg_query/grammar/statements/select.y" { (yyval.subquerytype) = PG_ANY_SUBLINK; ;} break; - case 1056: -#line 3299 "third_party/libpg_query/grammar/statements/select.y" + case 1062: +#line 3324 "third_party/libpg_query/grammar/statements/select.y" { (yyval.subquerytype) = PG_ALL_SUBLINK; ;} break; - case 1057: -#line 3302 "third_party/libpg_query/grammar/statements/select.y" + case 1063: +#line 3327 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1058: -#line 3303 "third_party/libpg_query/grammar/statements/select.y" + case 1064: +#line 3328 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) (yyvsp[(1) - (1)].conststr); ;} break; - case 1059: -#line 3306 "third_party/libpg_query/grammar/statements/select.y" + case 1065: +#line 3331 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "+"; ;} break; - case 1060: -#line 3307 "third_party/libpg_query/grammar/statements/select.y" + case 1066: +#line 3332 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "-"; ;} break; - case 1061: -#line 3308 "third_party/libpg_query/grammar/statements/select.y" + case 1067: +#line 3333 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "*"; ;} break; - case 1062: -#line 3309 "third_party/libpg_query/grammar/statements/select.y" + case 1068: +#line 3334 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "/"; ;} break; - case 1063: -#line 3310 "third_party/libpg_query/grammar/statements/select.y" + case 1069: +#line 3335 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "//"; ;} break; - case 1064: -#line 3311 "third_party/libpg_query/grammar/statements/select.y" + case 1070: +#line 3336 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "%"; ;} break; - case 1065: -#line 3312 "third_party/libpg_query/grammar/statements/select.y" + case 1071: +#line 3337 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "^"; ;} break; - case 1066: -#line 3313 "third_party/libpg_query/grammar/statements/select.y" + case 1072: +#line 3338 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "**"; ;} break; - case 1067: -#line 3314 "third_party/libpg_query/grammar/statements/select.y" + case 1073: +#line 3339 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "<"; ;} break; - case 1068: -#line 3315 "third_party/libpg_query/grammar/statements/select.y" + case 1074: +#line 3340 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = ">"; ;} break; - case 1069: -#line 3316 "third_party/libpg_query/grammar/statements/select.y" + case 1075: +#line 3341 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "="; ;} break; - case 1070: -#line 3317 "third_party/libpg_query/grammar/statements/select.y" + case 1076: +#line 3342 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "<="; ;} break; - case 1071: -#line 3318 "third_party/libpg_query/grammar/statements/select.y" + case 1077: +#line 3343 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = ">="; ;} break; - case 1072: -#line 3319 "third_party/libpg_query/grammar/statements/select.y" + case 1078: +#line 3344 "third_party/libpg_query/grammar/statements/select.y" { (yyval.conststr) = "<>"; ;} break; - case 1073: -#line 3323 "third_party/libpg_query/grammar/statements/select.y" + case 1079: +#line 3348 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} break; - case 1074: -#line 3325 "third_party/libpg_query/grammar/statements/select.y" + case 1080: +#line 3350 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(3) - (4)].list); ;} break; - case 1075: -#line 3330 "third_party/libpg_query/grammar/statements/select.y" + case 1081: +#line 3355 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} break; - case 1076: -#line 3332 "third_party/libpg_query/grammar/statements/select.y" + case 1082: +#line 3357 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(3) - (4)].list); ;} break; - case 1077: -#line 3337 "third_party/libpg_query/grammar/statements/select.y" + case 1083: +#line 3362 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} break; - case 1078: -#line 3339 "third_party/libpg_query/grammar/statements/select.y" + case 1084: +#line 3364 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(3) - (4)].list); ;} break; - case 1079: -#line 3341 "third_party/libpg_query/grammar/statements/select.y" + case 1085: +#line 3366 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString("~~")); ;} break; - case 1080: -#line 3343 "third_party/libpg_query/grammar/statements/select.y" + case 1086: +#line 3368 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString("!~~")); ;} break; - case 1081: -#line 3345 "third_party/libpg_query/grammar/statements/select.y" + case 1087: +#line 3370 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString("~~~")); ;} break; - case 1082: -#line 3347 "third_party/libpg_query/grammar/statements/select.y" + case 1088: +#line 3372 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString("!~~~")); ;} break; - case 1083: -#line 3349 "third_party/libpg_query/grammar/statements/select.y" + case 1089: +#line 3374 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString("~~*")); ;} break; - case 1084: -#line 3351 "third_party/libpg_query/grammar/statements/select.y" + case 1090: +#line 3376 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString("!~~*")); ;} break; - case 1085: -#line 3365 "third_party/libpg_query/grammar/statements/select.y" + case 1091: +#line 3390 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} break; - case 1086: -#line 3367 "third_party/libpg_query/grammar/statements/select.y" + case 1092: +#line 3392 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lcons(makeString((yyvsp[(1) - (3)].str)), (yyvsp[(3) - (3)].list)); ;} break; - case 1087: -#line 3372 "third_party/libpg_query/grammar/statements/select.y" + case 1093: +#line 3397 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} break; - case 1088: -#line 3376 "third_party/libpg_query/grammar/statements/select.y" + case 1094: +#line 3401 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} break; - case 1089: -#line 3383 "third_party/libpg_query/grammar/statements/select.y" + case 1095: +#line 3408 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1090: -#line 3388 "third_party/libpg_query/grammar/statements/select.y" + case 1096: +#line 3413 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (2)].list); ;} break; - case 1091: -#line 3394 "third_party/libpg_query/grammar/statements/select.y" + case 1097: +#line 3419 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} break; - case 1092: -#line 3398 "third_party/libpg_query/grammar/statements/select.y" + case 1098: +#line 3423 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} break; - case 1093: -#line 3405 "third_party/libpg_query/grammar/statements/select.y" + case 1099: +#line 3430 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1094: -#line 3410 "third_party/libpg_query/grammar/statements/select.y" + case 1100: +#line 3435 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (2)].list); ;} break; - case 1095: -#line 3417 "third_party/libpg_query/grammar/statements/select.y" + case 1101: +#line 3442 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1096: -#line 3421 "third_party/libpg_query/grammar/statements/select.y" + case 1102: +#line 3446 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NULL; ;} break; - case 1097: -#line 3430 "third_party/libpg_query/grammar/statements/select.y" + case 1103: +#line 3455 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} break; - case 1098: -#line 3434 "third_party/libpg_query/grammar/statements/select.y" + case 1104: +#line 3459 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} break; - case 1099: -#line 3440 "third_party/libpg_query/grammar/statements/select.y" + case 1105: +#line 3465 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = (yyvsp[(1) - (1)].node); ;} break; - case 1100: -#line 3444 "third_party/libpg_query/grammar/statements/select.y" + case 1106: +#line 3469 "third_party/libpg_query/grammar/statements/select.y" { PGNamedArgExpr *na = makeNode(PGNamedArgExpr); na->name = (yyvsp[(1) - (3)].str); @@ -27213,8 +27385,8 @@ YYLTYPE yylloc; ;} break; - case 1101: -#line 3453 "third_party/libpg_query/grammar/statements/select.y" + case 1107: +#line 3478 "third_party/libpg_query/grammar/statements/select.y" { PGNamedArgExpr *na = makeNode(PGNamedArgExpr); na->name = (yyvsp[(1) - (3)].str); @@ -27225,131 +27397,131 @@ YYLTYPE yylloc; ;} break; - case 1102: -#line 3463 "third_party/libpg_query/grammar/statements/select.y" + case 1108: +#line 3488 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].typnam)); ;} break; - case 1103: -#line 3464 "third_party/libpg_query/grammar/statements/select.y" + case 1109: +#line 3489 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].typnam)); ;} break; - case 1104: -#line 3469 "third_party/libpg_query/grammar/statements/select.y" + case 1110: +#line 3494 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make2(makeStringConst((yyvsp[(1) - (3)].str), (yylsp[(1) - (3)])), (yyvsp[(3) - (3)].node)); ;} break; - case 1105: -#line 3472 "third_party/libpg_query/grammar/statements/select.y" + case 1111: +#line 3497 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NIL; ;} break; - case 1106: -#line 3479 "third_party/libpg_query/grammar/statements/select.y" + case 1112: +#line 3504 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1107: -#line 3480 "third_party/libpg_query/grammar/statements/select.y" + case 1113: +#line 3505 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) "year"; ;} break; - case 1108: -#line 3481 "third_party/libpg_query/grammar/statements/select.y" + case 1114: +#line 3506 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) "month"; ;} break; - case 1109: -#line 3482 "third_party/libpg_query/grammar/statements/select.y" + case 1115: +#line 3507 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) "day"; ;} break; - case 1110: -#line 3483 "third_party/libpg_query/grammar/statements/select.y" + case 1116: +#line 3508 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) "hour"; ;} break; - case 1111: -#line 3484 "third_party/libpg_query/grammar/statements/select.y" + case 1117: +#line 3509 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) "minute"; ;} break; - case 1112: -#line 3485 "third_party/libpg_query/grammar/statements/select.y" + case 1118: +#line 3510 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) "second"; ;} break; - case 1113: -#line 3486 "third_party/libpg_query/grammar/statements/select.y" + case 1119: +#line 3511 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) "millisecond"; ;} break; - case 1114: -#line 3487 "third_party/libpg_query/grammar/statements/select.y" + case 1120: +#line 3512 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (char*) "microsecond"; ;} break; - case 1115: -#line 3488 "third_party/libpg_query/grammar/statements/select.y" + case 1121: +#line 3513 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1116: -#line 3499 "third_party/libpg_query/grammar/statements/select.y" + case 1122: +#line 3524 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make4((yyvsp[(1) - (4)].node), (yyvsp[(2) - (4)].node), (yyvsp[(3) - (4)].node), (yyvsp[(4) - (4)].node)); ;} break; - case 1117: -#line 3503 "third_party/libpg_query/grammar/statements/select.y" + case 1123: +#line 3528 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make3((yyvsp[(1) - (3)].node), (yyvsp[(2) - (3)].node), (yyvsp[(3) - (3)].node)); ;} break; - case 1118: -#line 3510 "third_party/libpg_query/grammar/statements/select.y" + case 1124: +#line 3535 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = (yyvsp[(2) - (2)].node); ;} break; - case 1119: -#line 3516 "third_party/libpg_query/grammar/statements/select.y" + case 1125: +#line 3541 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make2((yyvsp[(3) - (3)].node), (yyvsp[(1) - (3)].node)); ;} break; - case 1120: -#line 3517 "third_party/libpg_query/grammar/statements/select.y" + case 1126: +#line 3542 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NIL; ;} break; - case 1121: -#line 3534 "third_party/libpg_query/grammar/statements/select.y" + case 1127: +#line 3559 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make3((yyvsp[(1) - (3)].node), (yyvsp[(2) - (3)].node), (yyvsp[(3) - (3)].node)); ;} break; - case 1122: -#line 3538 "third_party/libpg_query/grammar/statements/select.y" + case 1128: +#line 3563 "third_party/libpg_query/grammar/statements/select.y" { /* not legal per SQL99, but might as well allow it */ (yyval.list) = list_make3((yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yyvsp[(2) - (3)].node)); ;} break; - case 1123: -#line 3543 "third_party/libpg_query/grammar/statements/select.y" + case 1129: +#line 3568 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make2((yyvsp[(1) - (2)].node), (yyvsp[(2) - (2)].node)); ;} break; - case 1124: -#line 3547 "third_party/libpg_query/grammar/statements/select.y" + case 1130: +#line 3572 "third_party/libpg_query/grammar/statements/select.y" { /* * Since there are no cases where this syntax allows @@ -27366,45 +27538,45 @@ YYLTYPE yylloc; ;} break; - case 1125: -#line 3562 "third_party/libpg_query/grammar/statements/select.y" + case 1131: +#line 3587 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1126: -#line 3566 "third_party/libpg_query/grammar/statements/select.y" + case 1132: +#line 3591 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NIL; ;} break; - case 1127: -#line 3570 "third_party/libpg_query/grammar/statements/select.y" + case 1133: +#line 3595 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = (yyvsp[(2) - (2)].node); ;} break; - case 1128: -#line 3573 "third_party/libpg_query/grammar/statements/select.y" + case 1134: +#line 3598 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = (yyvsp[(2) - (2)].node); ;} break; - case 1129: -#line 3576 "third_party/libpg_query/grammar/statements/select.y" + case 1135: +#line 3601 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(3) - (3)].list), (yyvsp[(1) - (3)].node)); ;} break; - case 1130: -#line 3577 "third_party/libpg_query/grammar/statements/select.y" + case 1136: +#line 3602 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(2) - (2)].list); ;} break; - case 1131: -#line 3578 "third_party/libpg_query/grammar/statements/select.y" + case 1137: +#line 3603 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1132: -#line 3582 "third_party/libpg_query/grammar/statements/select.y" + case 1138: +#line 3607 "third_party/libpg_query/grammar/statements/select.y" { PGSubLink *n = makeNode(PGSubLink); n->subselect = (yyvsp[(1) - (1)].node); @@ -27413,13 +27585,13 @@ YYLTYPE yylloc; ;} break; - case 1133: -#line 3588 "third_party/libpg_query/grammar/statements/select.y" + case 1139: +#line 3613 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = (PGNode *)(yyvsp[(2) - (3)].list); ;} break; - case 1134: -#line 3599 "third_party/libpg_query/grammar/statements/select.y" + case 1140: +#line 3624 "third_party/libpg_query/grammar/statements/select.y" { PGCaseExpr *c = makeNode(PGCaseExpr); c->casetype = InvalidOid; /* not analyzed yet */ @@ -27431,18 +27603,18 @@ YYLTYPE yylloc; ;} break; - case 1135: -#line 3612 "third_party/libpg_query/grammar/statements/select.y" + case 1141: +#line 3637 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} break; - case 1136: -#line 3613 "third_party/libpg_query/grammar/statements/select.y" + case 1142: +#line 3638 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} break; - case 1137: -#line 3618 "third_party/libpg_query/grammar/statements/select.y" + case 1143: +#line 3643 "third_party/libpg_query/grammar/statements/select.y" { PGCaseWhen *w = makeNode(PGCaseWhen); w->expr = (PGExpr *) (yyvsp[(2) - (4)].node); @@ -27452,42 +27624,42 @@ YYLTYPE yylloc; ;} break; - case 1138: -#line 3628 "third_party/libpg_query/grammar/statements/select.y" + case 1144: +#line 3653 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = (yyvsp[(2) - (2)].node); ;} break; - case 1139: -#line 3629 "third_party/libpg_query/grammar/statements/select.y" + case 1145: +#line 3654 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = NULL; ;} break; - case 1140: -#line 3632 "third_party/libpg_query/grammar/statements/select.y" + case 1146: +#line 3657 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = (yyvsp[(1) - (1)].node); ;} break; - case 1141: -#line 3633 "third_party/libpg_query/grammar/statements/select.y" + case 1147: +#line 3658 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = NULL; ;} break; - case 1142: -#line 3637 "third_party/libpg_query/grammar/statements/select.y" + case 1148: +#line 3662 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeColumnRef((yyvsp[(1) - (1)].str), NIL, (yylsp[(1) - (1)]), yyscanner); ;} break; - case 1143: -#line 3641 "third_party/libpg_query/grammar/statements/select.y" + case 1149: +#line 3666 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeColumnRef((yyvsp[(1) - (2)].str), (yyvsp[(2) - (2)].list), (yylsp[(1) - (2)]), yyscanner); ;} break; - case 1144: -#line 3648 "third_party/libpg_query/grammar/statements/select.y" + case 1150: +#line 3673 "third_party/libpg_query/grammar/statements/select.y" { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = false; @@ -27497,8 +27669,8 @@ YYLTYPE yylloc; ;} break; - case 1145: -#line 3656 "third_party/libpg_query/grammar/statements/select.y" + case 1151: +#line 3681 "third_party/libpg_query/grammar/statements/select.y" { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; @@ -27508,8 +27680,8 @@ YYLTYPE yylloc; ;} break; - case 1146: -#line 3663 "third_party/libpg_query/grammar/statements/select.y" + case 1152: +#line 3688 "third_party/libpg_query/grammar/statements/select.y" { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; @@ -27520,8 +27692,8 @@ YYLTYPE yylloc; ;} break; - case 1147: -#line 3671 "third_party/libpg_query/grammar/statements/select.y" + case 1153: +#line 3696 "third_party/libpg_query/grammar/statements/select.y" { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; @@ -27531,43 +27703,43 @@ YYLTYPE yylloc; ;} break; - case 1148: -#line 3681 "third_party/libpg_query/grammar/statements/select.y" + case 1154: +#line 3706 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = (yyvsp[(1) - (1)].node); ;} break; - case 1149: -#line 3682 "third_party/libpg_query/grammar/statements/select.y" + case 1155: +#line 3707 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = NULL; ;} break; - case 1150: -#line 3687 "third_party/libpg_query/grammar/statements/select.y" + case 1156: +#line 3712 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NIL; ;} break; - case 1151: -#line 3688 "third_party/libpg_query/grammar/statements/select.y" + case 1157: +#line 3713 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} break; - case 1152: -#line 3692 "third_party/libpg_query/grammar/statements/select.y" + case 1158: +#line 3717 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NULL; ;} break; - case 1153: -#line 3693 "third_party/libpg_query/grammar/statements/select.y" + case 1159: +#line 3718 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(NULL); ;} break; - case 1154: -#line 3694 "third_party/libpg_query/grammar/statements/select.y" + case 1160: +#line 3719 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(2) - (3)].list); ;} break; - case 1155: -#line 3699 "third_party/libpg_query/grammar/statements/select.y" + case 1161: +#line 3724 "third_party/libpg_query/grammar/statements/select.y" { if ((yyvsp[(3) - (3)].list)) { PGFuncCall *n = makeFuncCall(list_make1(makeString((yyvsp[(2) - (3)].str))), (yyvsp[(3) - (3)].list)->head->data.ptr_value ? (yyvsp[(3) - (3)].list) : NULL, (yylsp[(2) - (3)])); @@ -27578,8 +27750,8 @@ YYLTYPE yylloc; ;} break; - case 1156: -#line 3708 "third_party/libpg_query/grammar/statements/select.y" + case 1162: +#line 3733 "third_party/libpg_query/grammar/statements/select.y" { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = false; @@ -27589,8 +27761,8 @@ YYLTYPE yylloc; ;} break; - case 1157: -#line 3716 "third_party/libpg_query/grammar/statements/select.y" + case 1163: +#line 3741 "third_party/libpg_query/grammar/statements/select.y" { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; @@ -27600,8 +27772,8 @@ YYLTYPE yylloc; ;} break; - case 1158: -#line 3723 "third_party/libpg_query/grammar/statements/select.y" + case 1164: +#line 3748 "third_party/libpg_query/grammar/statements/select.y" { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; @@ -27612,8 +27784,8 @@ YYLTYPE yylloc; ;} break; - case 1159: -#line 3732 "third_party/libpg_query/grammar/statements/select.y" + case 1165: +#line 3757 "third_party/libpg_query/grammar/statements/select.y" { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; @@ -27623,48 +27795,48 @@ YYLTYPE yylloc; ;} break; - case 1160: -#line 3747 "third_party/libpg_query/grammar/statements/select.y" + case 1166: +#line 3772 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NIL; ;} break; - case 1161: -#line 3748 "third_party/libpg_query/grammar/statements/select.y" + case 1167: +#line 3773 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} break; - case 1164: -#line 3764 "third_party/libpg_query/grammar/statements/select.y" + case 1170: +#line 3789 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1165: -#line 3765 "third_party/libpg_query/grammar/statements/select.y" + case 1171: +#line 3790 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NIL; ;} break; - case 1166: -#line 3769 "third_party/libpg_query/grammar/statements/select.y" + case 1172: +#line 3794 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].target)); ;} break; - case 1167: -#line 3770 "third_party/libpg_query/grammar/statements/select.y" + case 1173: +#line 3795 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].target)); ;} break; - case 1168: -#line 3774 "third_party/libpg_query/grammar/statements/select.y" + case 1174: +#line 3799 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1169: -#line 3775 "third_party/libpg_query/grammar/statements/select.y" + case 1175: +#line 3800 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (2)].list); ;} break; - case 1170: -#line 3779 "third_party/libpg_query/grammar/statements/select.y" + case 1176: +#line 3804 "third_party/libpg_query/grammar/statements/select.y" { (yyval.target) = makeNode(PGResTarget); (yyval.target)->name = (yyvsp[(3) - (3)].str); @@ -27674,8 +27846,8 @@ YYLTYPE yylloc; ;} break; - case 1171: -#line 3795 "third_party/libpg_query/grammar/statements/select.y" + case 1177: +#line 3820 "third_party/libpg_query/grammar/statements/select.y" { (yyval.target) = makeNode(PGResTarget); (yyval.target)->name = (yyvsp[(2) - (2)].str); @@ -27685,8 +27857,8 @@ YYLTYPE yylloc; ;} break; - case 1172: -#line 3803 "third_party/libpg_query/grammar/statements/select.y" + case 1178: +#line 3828 "third_party/libpg_query/grammar/statements/select.y" { (yyval.target) = makeNode(PGResTarget); (yyval.target)->name = NULL; @@ -27696,140 +27868,140 @@ YYLTYPE yylloc; ;} break; - case 1173: -#line 3812 "third_party/libpg_query/grammar/statements/select.y" + case 1179: +#line 3837 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(3) - (4)].list); ;} break; - case 1174: -#line 3813 "third_party/libpg_query/grammar/statements/select.y" + case 1180: +#line 3838 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(2) - (2)].str))); ;} break; - case 1175: -#line 3816 "third_party/libpg_query/grammar/statements/select.y" + case 1181: +#line 3841 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1176: -#line 3817 "third_party/libpg_query/grammar/statements/select.y" + case 1182: +#line 3842 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NULL; ;} break; - case 1177: -#line 3820 "third_party/libpg_query/grammar/statements/select.y" + case 1183: +#line 3845 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make2((yyvsp[(1) - (3)].node), makeString((yyvsp[(3) - (3)].str))); ;} break; - case 1178: -#line 3824 "third_party/libpg_query/grammar/statements/select.y" + case 1184: +#line 3849 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].list)); ;} break; - case 1179: -#line 3825 "third_party/libpg_query/grammar/statements/select.y" + case 1185: +#line 3850 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].list)); ;} break; - case 1180: -#line 3829 "third_party/libpg_query/grammar/statements/select.y" + case 1186: +#line 3854 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1181: -#line 3830 "third_party/libpg_query/grammar/statements/select.y" + case 1187: +#line 3855 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (2)].list); ;} break; - case 1182: -#line 3833 "third_party/libpg_query/grammar/statements/select.y" + case 1188: +#line 3858 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(3) - (4)].list); ;} break; - case 1183: -#line 3834 "third_party/libpg_query/grammar/statements/select.y" + case 1189: +#line 3859 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(2) - (2)].list)); ;} break; - case 1184: -#line 3835 "third_party/libpg_query/grammar/statements/select.y" + case 1190: +#line 3860 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NULL; ;} break; - case 1185: -#line 3845 "third_party/libpg_query/grammar/statements/select.y" + case 1191: +#line 3870 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].range)); ;} break; - case 1186: -#line 3846 "third_party/libpg_query/grammar/statements/select.y" + case 1192: +#line 3871 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].range)); ;} break; - case 1187: -#line 3851 "third_party/libpg_query/grammar/statements/select.y" + case 1193: +#line 3876 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} break; - case 1188: -#line 3853 "third_party/libpg_query/grammar/statements/select.y" + case 1194: +#line 3878 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), makeString((yyvsp[(3) - (3)].str))); ;} break; - case 1189: -#line 3858 "third_party/libpg_query/grammar/statements/select.y" + case 1195: +#line 3883 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1190: -#line 3859 "third_party/libpg_query/grammar/statements/select.y" + case 1196: +#line 3884 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (2)].list); ;} break; - case 1191: -#line 3863 "third_party/libpg_query/grammar/statements/select.y" + case 1197: +#line 3888 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(1) - (1)].list); ;} break; - case 1192: -#line 3864 "third_party/libpg_query/grammar/statements/select.y" + case 1198: +#line 3889 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(2) - (3)].list); ;} break; - case 1193: -#line 3867 "third_party/libpg_query/grammar/statements/select.y" + case 1199: +#line 3892 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1194: -#line 3879 "third_party/libpg_query/grammar/statements/select.y" + case 1200: +#line 3904 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} break; - case 1195: -#line 3882 "third_party/libpg_query/grammar/statements/select.y" + case 1201: +#line 3907 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = check_func_name(lcons(makeString((yyvsp[(1) - (2)].str)), (yyvsp[(2) - (2)].list)), yyscanner); ;} break; - case 1196: -#line 3893 "third_party/libpg_query/grammar/statements/select.y" + case 1202: +#line 3918 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeIntConst((yyvsp[(1) - (1)].ival), (yylsp[(1) - (1)])); ;} break; - case 1197: -#line 3897 "third_party/libpg_query/grammar/statements/select.y" + case 1203: +#line 3922 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeFloatConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); ;} break; - case 1198: -#line 3901 "third_party/libpg_query/grammar/statements/select.y" + case 1204: +#line 3926 "third_party/libpg_query/grammar/statements/select.y" { if ((yyvsp[(2) - (2)].list)) { @@ -27843,15 +28015,15 @@ YYLTYPE yylloc; ;} break; - case 1199: -#line 3913 "third_party/libpg_query/grammar/statements/select.y" + case 1205: +#line 3938 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeBitStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); ;} break; - case 1200: -#line 3917 "third_party/libpg_query/grammar/statements/select.y" + case 1206: +#line 3942 "third_party/libpg_query/grammar/statements/select.y" { /* This is a bit constant per SQL99: * Without Feature F511, "BIT data type", @@ -27862,8 +28034,8 @@ YYLTYPE yylloc; ;} break; - case 1201: -#line 3926 "third_party/libpg_query/grammar/statements/select.y" + case 1207: +#line 3951 "third_party/libpg_query/grammar/statements/select.y" { /* generic type 'literal' syntax */ PGTypeName *t = makeTypeNameFromNameList((yyvsp[(1) - (2)].list)); @@ -27872,8 +28044,8 @@ YYLTYPE yylloc; ;} break; - case 1202: -#line 3933 "third_party/libpg_query/grammar/statements/select.y" + case 1208: +#line 3958 "third_party/libpg_query/grammar/statements/select.y" { /* generic syntax with a type modifier */ PGTypeName *t = makeTypeNameFromNameList((yyvsp[(1) - (7)].list)); @@ -27913,146 +28085,146 @@ YYLTYPE yylloc; ;} break; - case 1203: -#line 3971 "third_party/libpg_query/grammar/statements/select.y" + case 1209: +#line 3996 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeStringConstCast((yyvsp[(2) - (2)].str), (yylsp[(2) - (2)]), (yyvsp[(1) - (2)].typnam)); ;} break; - case 1204: -#line 3975 "third_party/libpg_query/grammar/statements/select.y" + case 1210: +#line 4000 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeIntervalNode((yyvsp[(3) - (5)].node), (yylsp[(3) - (5)]), (yyvsp[(5) - (5)].list)); ;} break; - case 1205: -#line 3979 "third_party/libpg_query/grammar/statements/select.y" + case 1211: +#line 4004 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeIntervalNode((yyvsp[(2) - (3)].ival), (yylsp[(2) - (3)]), (yyvsp[(3) - (3)].list)); ;} break; - case 1206: -#line 3983 "third_party/libpg_query/grammar/statements/select.y" + case 1212: +#line 4008 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeIntervalNode((yyvsp[(2) - (3)].str), (yylsp[(2) - (3)]), (yyvsp[(3) - (3)].list)); ;} break; - case 1207: -#line 3987 "third_party/libpg_query/grammar/statements/select.y" + case 1213: +#line 4012 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeBoolAConst(true, (yylsp[(1) - (1)])); ;} break; - case 1208: -#line 3991 "third_party/libpg_query/grammar/statements/select.y" + case 1214: +#line 4016 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeBoolAConst(false, (yylsp[(1) - (1)])); ;} break; - case 1209: -#line 3995 "third_party/libpg_query/grammar/statements/select.y" + case 1215: +#line 4020 "third_party/libpg_query/grammar/statements/select.y" { (yyval.node) = makeNullAConst((yylsp[(1) - (1)])); ;} break; - case 1210: -#line 4000 "third_party/libpg_query/grammar/statements/select.y" + case 1216: +#line 4025 "third_party/libpg_query/grammar/statements/select.y" { (yyval.ival) = (yyvsp[(1) - (1)].ival); ;} break; - case 1211: -#line 4017 "third_party/libpg_query/grammar/statements/select.y" + case 1217: +#line 4042 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1212: -#line 4018 "third_party/libpg_query/grammar/statements/select.y" + case 1218: +#line 4043 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} break; - case 1213: -#line 4019 "third_party/libpg_query/grammar/statements/select.y" + case 1219: +#line 4044 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} break; - case 1214: -#line 4022 "third_party/libpg_query/grammar/statements/select.y" + case 1220: +#line 4047 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1215: -#line 4023 "third_party/libpg_query/grammar/statements/select.y" + case 1221: +#line 4048 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} break; - case 1216: -#line 4024 "third_party/libpg_query/grammar/statements/select.y" + case 1222: +#line 4049 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} break; - case 1217: -#line 4027 "third_party/libpg_query/grammar/statements/select.y" + case 1223: +#line 4052 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1218: -#line 4028 "third_party/libpg_query/grammar/statements/select.y" + case 1224: +#line 4053 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} break; - case 1219: -#line 4029 "third_party/libpg_query/grammar/statements/select.y" + case 1225: +#line 4054 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} break; - case 1220: -#line 4032 "third_party/libpg_query/grammar/statements/select.y" + case 1226: +#line 4057 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} break; - case 1221: -#line 4033 "third_party/libpg_query/grammar/statements/select.y" + case 1227: +#line 4058 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lcons(makeString((yyvsp[(1) - (2)].str)), (yyvsp[(2) - (2)].list)); ;} break; - case 1222: -#line 4037 "third_party/libpg_query/grammar/statements/select.y" + case 1228: +#line 4062 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = list_make1(makeString((yyvsp[(2) - (2)].str))); ;} break; - case 1223: -#line 4039 "third_party/libpg_query/grammar/statements/select.y" + case 1229: +#line 4064 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), makeString((yyvsp[(3) - (3)].str))); ;} break; - case 1224: -#line 4043 "third_party/libpg_query/grammar/statements/select.y" + case 1230: +#line 4068 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = (yyvsp[(2) - (3)].list); ;} break; - case 1225: -#line 4044 "third_party/libpg_query/grammar/statements/select.y" + case 1231: +#line 4069 "third_party/libpg_query/grammar/statements/select.y" { (yyval.list) = NIL; ;} break; - case 1227: -#line 4051 "third_party/libpg_query/grammar/statements/select.y" + case 1233: +#line 4076 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1228: -#line 4052 "third_party/libpg_query/grammar/statements/select.y" + case 1234: +#line 4077 "third_party/libpg_query/grammar/statements/select.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1229: + case 1235: #line 8 "third_party/libpg_query/grammar/statements/prepare.y" { PGPrepareStmt *n = makeNode(PGPrepareStmt); @@ -28063,17 +28235,17 @@ YYLTYPE yylloc; ;} break; - case 1230: + case 1236: #line 18 "third_party/libpg_query/grammar/statements/prepare.y" { (yyval.list) = (yyvsp[(2) - (3)].list); ;} break; - case 1231: + case 1237: #line 19 "third_party/libpg_query/grammar/statements/prepare.y" { (yyval.list) = NIL; ;} break; - case 1237: + case 1243: #line 8 "third_party/libpg_query/grammar/statements/create_schema.y" { PGCreateSchemaStmt *n = makeNode(PGCreateSchemaStmt); @@ -28095,7 +28267,7 @@ YYLTYPE yylloc; ;} break; - case 1238: + case 1244: #line 27 "third_party/libpg_query/grammar/statements/create_schema.y" { PGCreateSchemaStmt *n = makeNode(PGCreateSchemaStmt); @@ -28122,7 +28294,7 @@ YYLTYPE yylloc; ;} break; - case 1239: + case 1245: #line 51 "third_party/libpg_query/grammar/statements/create_schema.y" { PGCreateSchemaStmt *n = makeNode(PGCreateSchemaStmt); @@ -28144,7 +28316,7 @@ YYLTYPE yylloc; ;} break; - case 1240: + case 1246: #line 74 "third_party/libpg_query/grammar/statements/create_schema.y" { if ((yyloc) < 0) /* see comments for YYLLOC_DEFAULT */ @@ -28153,12 +28325,12 @@ YYLTYPE yylloc; ;} break; - case 1241: + case 1247: #line 80 "third_party/libpg_query/grammar/statements/create_schema.y" { (yyval.list) = NIL; ;} break; - case 1246: + case 1252: #line 11 "third_party/libpg_query/grammar/statements/index.y" { PGIndexStmt *n = makeNode(PGIndexStmt); @@ -28184,7 +28356,7 @@ YYLTYPE yylloc; ;} break; - case 1247: + case 1253: #line 36 "third_party/libpg_query/grammar/statements/index.y" { PGIndexStmt *n = makeNode(PGIndexStmt); @@ -28210,62 +28382,62 @@ YYLTYPE yylloc; ;} break; - case 1248: + case 1254: #line 62 "third_party/libpg_query/grammar/statements/index.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1249: + case 1255: #line 66 "third_party/libpg_query/grammar/statements/index.y" { (yyval.str) = (yyvsp[(2) - (2)].str); ;} break; - case 1250: + case 1256: #line 67 "third_party/libpg_query/grammar/statements/index.y" { (yyval.str) = (char*) DEFAULT_INDEX_TYPE; ;} break; - case 1251: + case 1257: #line 72 "third_party/libpg_query/grammar/statements/index.y" { (yyval.boolean) = true; ;} break; - case 1252: + case 1258: #line 73 "third_party/libpg_query/grammar/statements/index.y" { (yyval.boolean) = false; ;} break; - case 1253: + case 1259: #line 78 "third_party/libpg_query/grammar/statements/index.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1254: + case 1260: #line 79 "third_party/libpg_query/grammar/statements/index.y" { (yyval.str) = NULL; ;} break; - case 1255: + case 1261: #line 83 "third_party/libpg_query/grammar/statements/index.y" { (yyval.list) = (yyvsp[(2) - (2)].list); ;} break; - case 1256: + case 1262: #line 84 "third_party/libpg_query/grammar/statements/index.y" { (yyval.list) = NIL; ;} break; - case 1257: + case 1263: #line 89 "third_party/libpg_query/grammar/statements/index.y" { (yyval.boolean) = true; ;} break; - case 1258: + case 1264: #line 90 "third_party/libpg_query/grammar/statements/index.y" { (yyval.boolean) = false; ;} break; - case 1259: + case 1265: #line 8 "third_party/libpg_query/grammar/statements/alter_schema.y" { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); @@ -28277,7 +28449,7 @@ YYLTYPE yylloc; ;} break; - case 1260: + case 1266: #line 17 "third_party/libpg_query/grammar/statements/alter_schema.y" { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); @@ -28289,7 +28461,7 @@ YYLTYPE yylloc; ;} break; - case 1261: + case 1267: #line 26 "third_party/libpg_query/grammar/statements/alter_schema.y" { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); @@ -28301,7 +28473,7 @@ YYLTYPE yylloc; ;} break; - case 1262: + case 1268: #line 35 "third_party/libpg_query/grammar/statements/alter_schema.y" { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); @@ -28313,7 +28485,7 @@ YYLTYPE yylloc; ;} break; - case 1263: + case 1269: #line 44 "third_party/libpg_query/grammar/statements/alter_schema.y" { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); @@ -28325,7 +28497,7 @@ YYLTYPE yylloc; ;} break; - case 1264: + case 1270: #line 53 "third_party/libpg_query/grammar/statements/alter_schema.y" { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); @@ -28337,7 +28509,7 @@ YYLTYPE yylloc; ;} break; - case 1265: + case 1271: #line 6 "third_party/libpg_query/grammar/statements/checkpoint.y" { PGCheckPointStmt *n = makeNode(PGCheckPointStmt); @@ -28347,7 +28519,7 @@ YYLTYPE yylloc; ;} break; - case 1266: + case 1272: #line 13 "third_party/libpg_query/grammar/statements/checkpoint.y" { PGCheckPointStmt *n = makeNode(PGCheckPointStmt); @@ -28357,17 +28529,17 @@ YYLTYPE yylloc; ;} break; - case 1267: + case 1273: #line 22 "third_party/libpg_query/grammar/statements/checkpoint.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1268: + case 1274: #line 23 "third_party/libpg_query/grammar/statements/checkpoint.y" { (yyval.str) = NULL; ;} break; - case 1269: + case 1275: #line 8 "third_party/libpg_query/grammar/statements/export.y" { PGExportStmt *n = makeNode(PGExportStmt); @@ -28381,7 +28553,7 @@ YYLTYPE yylloc; ;} break; - case 1270: + case 1276: #line 20 "third_party/libpg_query/grammar/statements/export.y" { PGExportStmt *n = makeNode(PGExportStmt); @@ -28395,7 +28567,7 @@ YYLTYPE yylloc; ;} break; - case 1271: + case 1277: #line 34 "third_party/libpg_query/grammar/statements/export.y" { PGImportStmt *n = makeNode(PGImportStmt); @@ -28404,7 +28576,7 @@ YYLTYPE yylloc; ;} break; - case 1272: + case 1278: #line 10 "third_party/libpg_query/grammar/statements/explain.y" { PGExplainStmt *n = makeNode(PGExplainStmt); @@ -28414,7 +28586,7 @@ YYLTYPE yylloc; ;} break; - case 1273: + case 1279: #line 17 "third_party/libpg_query/grammar/statements/explain.y" { PGExplainStmt *n = makeNode(PGExplainStmt); @@ -28427,7 +28599,7 @@ YYLTYPE yylloc; ;} break; - case 1274: + case 1280: #line 27 "third_party/libpg_query/grammar/statements/explain.y" { PGExplainStmt *n = makeNode(PGExplainStmt); @@ -28437,7 +28609,7 @@ YYLTYPE yylloc; ;} break; - case 1275: + case 1281: #line 34 "third_party/libpg_query/grammar/statements/explain.y" { PGExplainStmt *n = makeNode(PGExplainStmt); @@ -28447,118 +28619,118 @@ YYLTYPE yylloc; ;} break; - case 1276: + case 1282: #line 44 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.boolean) = true; ;} break; - case 1277: + case 1283: #line 45 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.boolean) = false; ;} break; - case 1278: + case 1284: #line 50 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.node) = (PGNode *) makeString((yyvsp[(1) - (1)].str)); ;} break; - case 1279: + case 1285: #line 51 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.node) = (PGNode *) (yyvsp[(1) - (1)].value); ;} break; - case 1280: + case 1286: #line 52 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.node) = NULL; ;} break; - case 1311: + case 1317: #line 90 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1312: + case 1318: #line 91 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} break; - case 1313: + case 1319: #line 92 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} break; - case 1314: + case 1320: #line 97 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1315: + case 1321: #line 98 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1316: + case 1322: #line 104 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].defelt)); ;} break; - case 1317: + case 1323: #line 108 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].defelt)); ;} break; - case 1318: + case 1324: #line 115 "third_party/libpg_query/grammar/statements/explain.y" {;} break; - case 1319: + case 1325: #line 116 "third_party/libpg_query/grammar/statements/explain.y" {;} break; - case 1320: + case 1326: #line 121 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (char*) "true"; ;} break; - case 1321: + case 1327: #line 122 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (char*) "false"; ;} break; - case 1322: + case 1328: #line 123 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (char*) "on"; ;} break; - case 1323: + case 1329: #line 129 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1324: + case 1330: #line 135 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.defelt) = makeDefElem((yyvsp[(1) - (2)].str), (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} break; - case 1325: + case 1331: #line 142 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1326: + case 1332: #line 143 "third_party/libpg_query/grammar/statements/explain.y" { (yyval.str) = (char*) "analyze"; ;} break; - case 1327: + case 1333: #line 11 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = (yyvsp[(2) - (2)].vsetstmt); @@ -28567,7 +28739,7 @@ YYLTYPE yylloc; ;} break; - case 1328: + case 1334: #line 17 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = (yyvsp[(3) - (3)].vsetstmt); @@ -28576,7 +28748,7 @@ YYLTYPE yylloc; ;} break; - case 1329: + case 1335: #line 23 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = (yyvsp[(3) - (3)].vsetstmt); @@ -28585,7 +28757,7 @@ YYLTYPE yylloc; ;} break; - case 1330: + case 1336: #line 29 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = (yyvsp[(3) - (3)].vsetstmt); @@ -28594,12 +28766,12 @@ YYLTYPE yylloc; ;} break; - case 1331: + case 1337: #line 38 "third_party/libpg_query/grammar/statements/variable_set.y" {(yyval.vsetstmt) = (yyvsp[(1) - (1)].vsetstmt);;} break; - case 1332: + case 1338: #line 40 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -28609,7 +28781,7 @@ YYLTYPE yylloc; ;} break; - case 1333: + case 1339: #line 48 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -28623,7 +28795,7 @@ YYLTYPE yylloc; ;} break; - case 1334: + case 1340: #line 59 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -28634,7 +28806,7 @@ YYLTYPE yylloc; ;} break; - case 1335: + case 1341: #line 71 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -28645,7 +28817,7 @@ YYLTYPE yylloc; ;} break; - case 1336: + case 1342: #line 79 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -28656,7 +28828,7 @@ YYLTYPE yylloc; ;} break; - case 1337: + case 1343: #line 87 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -28666,7 +28838,7 @@ YYLTYPE yylloc; ;} break; - case 1338: + case 1344: #line 94 "third_party/libpg_query/grammar/statements/variable_set.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -28676,31 +28848,31 @@ YYLTYPE yylloc; ;} break; - case 1339: + case 1345: #line 104 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.node) = makeStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); ;} break; - case 1340: + case 1346: #line 106 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.node) = makeAConst((yyvsp[(1) - (1)].value), (yylsp[(1) - (1)])); ;} break; - case 1341: + case 1347: #line 112 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.node) = makeStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); ;} break; - case 1342: + case 1348: #line 116 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.node) = makeStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); ;} break; - case 1343: + case 1349: #line 120 "third_party/libpg_query/grammar/statements/variable_set.y" { PGTypeName *t = (yyvsp[(1) - (3)].typnam); @@ -28718,7 +28890,7 @@ YYLTYPE yylloc; ;} break; - case 1344: + case 1350: #line 135 "third_party/libpg_query/grammar/statements/variable_set.y" { PGTypeName *t = (yyvsp[(1) - (5)].typnam); @@ -28728,32 +28900,32 @@ YYLTYPE yylloc; ;} break; - case 1345: + case 1351: #line 141 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.node) = makeAConst((yyvsp[(1) - (1)].value), (yylsp[(1) - (1)])); ;} break; - case 1346: + case 1352: #line 142 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.node) = NULL; ;} break; - case 1347: + case 1353: #line 143 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.node) = NULL; ;} break; - case 1348: + case 1354: #line 147 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} break; - case 1349: + case 1355: #line 148 "third_party/libpg_query/grammar/statements/variable_set.y" { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} break; - case 1350: + case 1356: #line 8 "third_party/libpg_query/grammar/statements/load.y" { PGLoadStmt *n = makeNode(PGLoadStmt); @@ -28764,7 +28936,7 @@ YYLTYPE yylloc; ;} break; - case 1351: + case 1357: #line 15 "third_party/libpg_query/grammar/statements/load.y" { PGLoadStmt *n = makeNode(PGLoadStmt); @@ -28775,7 +28947,7 @@ YYLTYPE yylloc; ;} break; - case 1352: + case 1358: #line 22 "third_party/libpg_query/grammar/statements/load.y" { PGLoadStmt *n = makeNode(PGLoadStmt); @@ -28786,7 +28958,7 @@ YYLTYPE yylloc; ;} break; - case 1353: + case 1359: #line 29 "third_party/libpg_query/grammar/statements/load.y" { PGLoadStmt *n = makeNode(PGLoadStmt); @@ -28797,7 +28969,7 @@ YYLTYPE yylloc; ;} break; - case 1354: + case 1360: #line 36 "third_party/libpg_query/grammar/statements/load.y" { PGLoadStmt *n = makeNode(PGLoadStmt); @@ -28808,27 +28980,27 @@ YYLTYPE yylloc; ;} break; - case 1355: + case 1361: #line 45 "third_party/libpg_query/grammar/statements/load.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1356: + case 1362: #line 46 "third_party/libpg_query/grammar/statements/load.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1357: + case 1363: #line 48 "third_party/libpg_query/grammar/statements/load.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1358: + case 1364: #line 49 "third_party/libpg_query/grammar/statements/load.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1359: + case 1365: #line 9 "third_party/libpg_query/grammar/statements/vacuum.y" { PGVacuumStmt *n = makeNode(PGVacuumStmt); @@ -28845,7 +29017,7 @@ YYLTYPE yylloc; ;} break; - case 1360: + case 1366: #line 23 "third_party/libpg_query/grammar/statements/vacuum.y" { PGVacuumStmt *n = makeNode(PGVacuumStmt); @@ -28862,7 +29034,7 @@ YYLTYPE yylloc; ;} break; - case 1361: + case 1367: #line 37 "third_party/libpg_query/grammar/statements/vacuum.y" { PGVacuumStmt *n = (PGVacuumStmt *) (yyvsp[(5) - (5)].node); @@ -28877,7 +29049,7 @@ YYLTYPE yylloc; ;} break; - case 1362: + case 1368: #line 49 "third_party/libpg_query/grammar/statements/vacuum.y" { PGVacuumStmt *n = makeNode(PGVacuumStmt); @@ -28888,7 +29060,7 @@ YYLTYPE yylloc; ;} break; - case 1363: + case 1369: #line 57 "third_party/libpg_query/grammar/statements/vacuum.y" { PGVacuumStmt *n = makeNode(PGVacuumStmt); @@ -28901,27 +29073,27 @@ YYLTYPE yylloc; ;} break; - case 1364: + case 1370: #line 70 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.ival) = PG_VACOPT_ANALYZE; ;} break; - case 1365: + case 1371: #line 71 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.ival) = PG_VACOPT_VERBOSE; ;} break; - case 1366: + case 1372: #line 72 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.ival) = PG_VACOPT_FREEZE; ;} break; - case 1367: + case 1373: #line 73 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.ival) = PG_VACOPT_FULL; ;} break; - case 1368: + case 1374: #line 75 "third_party/libpg_query/grammar/statements/vacuum.y" { if (strcmp((yyvsp[(1) - (1)].str), "disable_page_skipping") == 0) @@ -28934,37 +29106,37 @@ YYLTYPE yylloc; ;} break; - case 1369: + case 1375: #line 87 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.boolean) = true; ;} break; - case 1370: + case 1376: #line 88 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.boolean) = false; ;} break; - case 1371: + case 1377: #line 93 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.ival) = (yyvsp[(1) - (1)].ival); ;} break; - case 1372: + case 1378: #line 94 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.ival) = (yyvsp[(1) - (3)].ival) | (yyvsp[(3) - (3)].ival); ;} break; - case 1373: + case 1379: #line 98 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.boolean) = true; ;} break; - case 1374: + case 1380: #line 99 "third_party/libpg_query/grammar/statements/vacuum.y" { (yyval.boolean) = false; ;} break; - case 1375: + case 1381: #line 9 "third_party/libpg_query/grammar/statements/delete.y" { PGDeleteStmt *n = makeNode(PGDeleteStmt); @@ -28977,7 +29149,7 @@ YYLTYPE yylloc; ;} break; - case 1376: + case 1382: #line 19 "third_party/libpg_query/grammar/statements/delete.y" { PGDeleteStmt *n = makeNode(PGDeleteStmt); @@ -28990,14 +29162,14 @@ YYLTYPE yylloc; ;} break; - case 1377: + case 1383: #line 32 "third_party/libpg_query/grammar/statements/delete.y" { (yyval.range) = (yyvsp[(1) - (1)].range); ;} break; - case 1378: + case 1384: #line 36 "third_party/libpg_query/grammar/statements/delete.y" { PGAlias *alias = makeNode(PGAlias); @@ -29007,7 +29179,7 @@ YYLTYPE yylloc; ;} break; - case 1379: + case 1385: #line 43 "third_party/libpg_query/grammar/statements/delete.y" { PGAlias *alias = makeNode(PGAlias); @@ -29017,27 +29189,27 @@ YYLTYPE yylloc; ;} break; - case 1380: + case 1386: #line 53 "third_party/libpg_query/grammar/statements/delete.y" { (yyval.node) = (yyvsp[(2) - (2)].node); ;} break; - case 1381: + case 1387: #line 54 "third_party/libpg_query/grammar/statements/delete.y" { (yyval.node) = NULL; ;} break; - case 1382: + case 1388: #line 60 "third_party/libpg_query/grammar/statements/delete.y" { (yyval.list) = (yyvsp[(2) - (2)].list); ;} break; - case 1383: + case 1389: #line 61 "third_party/libpg_query/grammar/statements/delete.y" { (yyval.list) = NIL; ;} break; - case 1384: + case 1390: #line 10 "third_party/libpg_query/grammar/statements/analyze.y" { PGVacuumStmt *n = makeNode(PGVacuumStmt); @@ -29050,7 +29222,7 @@ YYLTYPE yylloc; ;} break; - case 1385: + case 1391: #line 20 "third_party/libpg_query/grammar/statements/analyze.y" { PGVacuumStmt *n = makeNode(PGVacuumStmt); @@ -29063,7 +29235,7 @@ YYLTYPE yylloc; ;} break; - case 1386: + case 1392: #line 8 "third_party/libpg_query/grammar/statements/attach.y" { PGAttachStmt *n = makeNode(PGAttachStmt); @@ -29074,7 +29246,7 @@ YYLTYPE yylloc; ;} break; - case 1387: + case 1393: #line 19 "third_party/libpg_query/grammar/statements/attach.y" { PGDetachStmt *n = makeNode(PGDetachStmt); @@ -29084,7 +29256,7 @@ YYLTYPE yylloc; ;} break; - case 1388: + case 1394: #line 26 "third_party/libpg_query/grammar/statements/attach.y" { PGDetachStmt *n = makeNode(PGDetachStmt); @@ -29094,27 +29266,27 @@ YYLTYPE yylloc; ;} break; - case 1389: + case 1395: #line 34 "third_party/libpg_query/grammar/statements/attach.y" {;} break; - case 1390: + case 1396: #line 35 "third_party/libpg_query/grammar/statements/attach.y" {;} break; - case 1391: + case 1397: #line 39 "third_party/libpg_query/grammar/statements/attach.y" { (yyval.str) = (yyvsp[(2) - (2)].str); ;} break; - case 1392: + case 1398: #line 40 "third_party/libpg_query/grammar/statements/attach.y" { (yyval.str) = NULL; ;} break; - case 1393: + case 1399: #line 3 "third_party/libpg_query/grammar/statements/variable_reset.y" { (yyvsp[(2) - (2)].vsetstmt)->scope = VAR_SET_SCOPE_DEFAULT; @@ -29122,7 +29294,7 @@ YYLTYPE yylloc; ;} break; - case 1394: + case 1400: #line 8 "third_party/libpg_query/grammar/statements/variable_reset.y" { (yyvsp[(3) - (3)].vsetstmt)->scope = VAR_SET_SCOPE_LOCAL; @@ -29130,7 +29302,7 @@ YYLTYPE yylloc; ;} break; - case 1395: + case 1401: #line 13 "third_party/libpg_query/grammar/statements/variable_reset.y" { (yyvsp[(3) - (3)].vsetstmt)->scope = VAR_SET_SCOPE_SESSION; @@ -29138,7 +29310,7 @@ YYLTYPE yylloc; ;} break; - case 1396: + case 1402: #line 18 "third_party/libpg_query/grammar/statements/variable_reset.y" { (yyvsp[(3) - (3)].vsetstmt)->scope = VAR_SET_SCOPE_GLOBAL; @@ -29146,7 +29318,7 @@ YYLTYPE yylloc; ;} break; - case 1397: + case 1403: #line 27 "third_party/libpg_query/grammar/statements/variable_reset.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -29156,7 +29328,7 @@ YYLTYPE yylloc; ;} break; - case 1398: + case 1404: #line 34 "third_party/libpg_query/grammar/statements/variable_reset.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -29165,12 +29337,12 @@ YYLTYPE yylloc; ;} break; - case 1399: + case 1405: #line 43 "third_party/libpg_query/grammar/statements/variable_reset.y" { (yyval.vsetstmt) = (yyvsp[(1) - (1)].vsetstmt); ;} break; - case 1400: + case 1406: #line 45 "third_party/libpg_query/grammar/statements/variable_reset.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -29180,7 +29352,7 @@ YYLTYPE yylloc; ;} break; - case 1401: + case 1407: #line 52 "third_party/libpg_query/grammar/statements/variable_reset.y" { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); @@ -29190,7 +29362,7 @@ YYLTYPE yylloc; ;} break; - case 1402: + case 1408: #line 3 "third_party/libpg_query/grammar/statements/variable_show.y" { PGVariableShowSelectStmt *n = makeNode(PGVariableShowSelectStmt); @@ -29201,7 +29373,7 @@ YYLTYPE yylloc; ;} break; - case 1403: + case 1409: #line 10 "third_party/libpg_query/grammar/statements/variable_show.y" { PGVariableShowSelectStmt *n = makeNode(PGVariableShowSelectStmt); @@ -29212,7 +29384,7 @@ YYLTYPE yylloc; ;} break; - case 1404: + case 1410: #line 18 "third_party/libpg_query/grammar/statements/variable_show.y" { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); @@ -29222,7 +29394,7 @@ YYLTYPE yylloc; ;} break; - case 1405: + case 1411: #line 25 "third_party/libpg_query/grammar/statements/variable_show.y" { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); @@ -29232,7 +29404,7 @@ YYLTYPE yylloc; ;} break; - case 1406: + case 1412: #line 32 "third_party/libpg_query/grammar/statements/variable_show.y" { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); @@ -29242,7 +29414,7 @@ YYLTYPE yylloc; ;} break; - case 1407: + case 1413: #line 39 "third_party/libpg_query/grammar/statements/variable_show.y" { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); @@ -29252,7 +29424,7 @@ YYLTYPE yylloc; ;} break; - case 1408: + case 1414: #line 46 "third_party/libpg_query/grammar/statements/variable_show.y" { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); @@ -29262,7 +29434,7 @@ YYLTYPE yylloc; ;} break; - case 1409: + case 1415: #line 53 "third_party/libpg_query/grammar/statements/variable_show.y" { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); @@ -29272,27 +29444,27 @@ YYLTYPE yylloc; ;} break; - case 1414: + case 1420: #line 65 "third_party/libpg_query/grammar/statements/variable_show.y" { (yyval.str) = (yyvsp[(1) - (1)].str); ;} break; - case 1415: + case 1421: #line 67 "third_party/libpg_query/grammar/statements/variable_show.y" { (yyval.str) = psprintf("%s.%s", (yyvsp[(1) - (3)].str), (yyvsp[(3) - (3)].str)); ;} break; - case 1416: + case 1422: #line 70 "third_party/libpg_query/grammar/statements/variable_show.y" { (yyval.str) = psprintf("\"%s\"", (yyvsp[(1) - (1)].str)); ;} break; - case 1417: + case 1423: #line 72 "third_party/libpg_query/grammar/statements/variable_show.y" { (yyval.str) = psprintf("%s.\"%s\"", (yyvsp[(1) - (3)].str), (yyvsp[(3) - (3)].str)); ;} break; - case 1418: + case 1424: #line 7 "third_party/libpg_query/grammar/statements/call.y" { PGCallStmt *n = makeNode(PGCallStmt); @@ -29301,7 +29473,7 @@ YYLTYPE yylloc; ;} break; - case 1419: + case 1425: #line 10 "third_party/libpg_query/grammar/statements/view.y" { PGViewStmt *n = makeNode(PGViewStmt); @@ -29316,7 +29488,7 @@ YYLTYPE yylloc; ;} break; - case 1420: + case 1426: #line 23 "third_party/libpg_query/grammar/statements/view.y" { PGViewStmt *n = makeNode(PGViewStmt); @@ -29331,7 +29503,7 @@ YYLTYPE yylloc; ;} break; - case 1421: + case 1427: #line 36 "third_party/libpg_query/grammar/statements/view.y" { PGViewStmt *n = makeNode(PGViewStmt); @@ -29346,7 +29518,7 @@ YYLTYPE yylloc; ;} break; - case 1422: + case 1428: #line 49 "third_party/libpg_query/grammar/statements/view.y" { PGViewStmt *n = makeNode(PGViewStmt); @@ -29366,7 +29538,7 @@ YYLTYPE yylloc; ;} break; - case 1423: + case 1429: #line 67 "third_party/libpg_query/grammar/statements/view.y" { PGViewStmt *n = makeNode(PGViewStmt); @@ -29386,27 +29558,27 @@ YYLTYPE yylloc; ;} break; - case 1424: + case 1430: #line 87 "third_party/libpg_query/grammar/statements/view.y" { (yyval.viewcheckoption) = CASCADED_CHECK_OPTION; ;} break; - case 1425: + case 1431: #line 88 "third_party/libpg_query/grammar/statements/view.y" { (yyval.viewcheckoption) = CASCADED_CHECK_OPTION; ;} break; - case 1426: + case 1432: #line 89 "third_party/libpg_query/grammar/statements/view.y" { (yyval.viewcheckoption) = PG_LOCAL_CHECK_OPTION; ;} break; - case 1427: + case 1433: #line 90 "third_party/libpg_query/grammar/statements/view.y" { (yyval.viewcheckoption) = PG_NO_CHECK_OPTION; ;} break; - case 1428: + case 1434: #line 12 "third_party/libpg_query/grammar/statements/create_as.y" { PGCreateTableAsStmt *ctas = makeNode(PGCreateTableAsStmt); @@ -29422,7 +29594,7 @@ YYLTYPE yylloc; ;} break; - case 1429: + case 1435: #line 25 "third_party/libpg_query/grammar/statements/create_as.y" { PGCreateTableAsStmt *ctas = makeNode(PGCreateTableAsStmt); @@ -29438,7 +29610,7 @@ YYLTYPE yylloc; ;} break; - case 1430: + case 1436: #line 38 "third_party/libpg_query/grammar/statements/create_as.y" { PGCreateTableAsStmt *ctas = makeNode(PGCreateTableAsStmt); @@ -29454,22 +29626,22 @@ YYLTYPE yylloc; ;} break; - case 1431: + case 1437: #line 54 "third_party/libpg_query/grammar/statements/create_as.y" { (yyval.boolean) = true; ;} break; - case 1432: + case 1438: #line 55 "third_party/libpg_query/grammar/statements/create_as.y" { (yyval.boolean) = false; ;} break; - case 1433: + case 1439: #line 56 "third_party/libpg_query/grammar/statements/create_as.y" { (yyval.boolean) = true; ;} break; - case 1434: + case 1440: #line 62 "third_party/libpg_query/grammar/statements/create_as.y" { (yyval.into) = makeNode(PGIntoClause); @@ -29484,7 +29656,7 @@ YYLTYPE yylloc; /* Line 1267 of yacc.c. */ -#line 29488 "third_party/libpg_query/grammar/grammar_out.cpp" +#line 29660 "third_party/libpg_query/grammar/grammar_out.cpp" default: break; } YY_SYMBOL_PRINT ("-> $$ =", yyr1[yyn], &yyval, &yyloc); diff --git a/src/duckdb/ub_src_common_arrow_appender.cpp b/src/duckdb/ub_src_common_arrow_appender.cpp index e3adc18af..94af8f11f 100644 --- a/src/duckdb/ub_src_common_arrow_appender.cpp +++ b/src/duckdb/ub_src_common_arrow_appender.cpp @@ -1,5 +1,9 @@ #include "src/common/arrow/appender/bool_data.cpp" +#include "src/common/arrow/appender/list_data.cpp" + +#include "src/common/arrow/appender/map_data.cpp" + #include "src/common/arrow/appender/struct_data.cpp" #include "src/common/arrow/appender/union_data.cpp" diff --git a/src/duckdb/ub_src_core_functions.cpp b/src/duckdb/ub_src_core_functions.cpp index 85991d622..961929d47 100644 --- a/src/duckdb/ub_src_core_functions.cpp +++ b/src/duckdb/ub_src_core_functions.cpp @@ -2,3 +2,5 @@ #include "src/core_functions/function_list.cpp" +#include "src/core_functions/lambda_functions.cpp" + diff --git a/src/duckdb/ub_src_core_functions_scalar_array.cpp b/src/duckdb/ub_src_core_functions_scalar_array.cpp new file mode 100644 index 000000000..49ed0ec37 --- /dev/null +++ b/src/duckdb/ub_src_core_functions_scalar_array.cpp @@ -0,0 +1,4 @@ +#include "src/core_functions/scalar/array/array_value.cpp" + +#include "src/core_functions/scalar/array/array_functions.cpp" + diff --git a/src/duckdb/ub_src_core_functions_scalar_list.cpp b/src/duckdb/ub_src_core_functions_scalar_list.cpp index 0ac5bed76..faf1ea33d 100644 --- a/src/duckdb/ub_src_core_functions_scalar_list.cpp +++ b/src/duckdb/ub_src_core_functions_scalar_list.cpp @@ -4,9 +4,7 @@ #include "src/core_functions/scalar/list/list_aggregates.cpp" -#include "src/core_functions/scalar/list/list_lambdas.cpp" - -#include "src/core_functions/scalar/list/list_value.cpp" +#include "src/core_functions/scalar/list/list_filter.cpp" #include "src/core_functions/scalar/list/list_sort.cpp" @@ -16,5 +14,9 @@ #include "src/core_functions/scalar/list/list_inner_product.cpp" +#include "src/core_functions/scalar/list/list_transform.cpp" + +#include "src/core_functions/scalar/list/list_value.cpp" + #include "src/core_functions/scalar/list/range.cpp" diff --git a/src/duckdb/ub_src_function_cast.cpp b/src/duckdb/ub_src_function_cast.cpp index 6bb310896..750007a0e 100644 --- a/src/duckdb/ub_src_function_cast.cpp +++ b/src/duckdb/ub_src_function_cast.cpp @@ -1,3 +1,5 @@ +#include "src/function/cast/array_casts.cpp" + #include "src/function/cast/blob_cast.cpp" #include "src/function/cast/bit_cast.cpp" diff --git a/src/duckdb/ub_src_function_scalar_list.cpp b/src/duckdb/ub_src_function_scalar_list.cpp index 3a2143078..2178ed155 100644 --- a/src/duckdb/ub_src_function_scalar_list.cpp +++ b/src/duckdb/ub_src_function_scalar_list.cpp @@ -6,3 +6,7 @@ #include "src/function/scalar/list/list_resize.cpp" +#include "src/function/scalar/list/list_zip.cpp" + +#include "src/function/scalar/list/list_select.cpp" + diff --git a/src/duckdb/ub_src_function_table_system.cpp b/src/duckdb/ub_src_function_table_system.cpp index 11a8b99a5..fd325e19e 100644 --- a/src/duckdb/ub_src_function_table_system.cpp +++ b/src/duckdb/ub_src_function_table_system.cpp @@ -38,8 +38,6 @@ #include "src/function/table/system/pragma_table_info.cpp" -#include "src/function/table/system/pragma_user_agent.cpp" - #include "src/function/table/system/test_all_types.cpp" #include "src/function/table/system/test_vector_types.cpp" diff --git a/src/duckdb/ub_src_parser_expression.cpp b/src/duckdb/ub_src_parser_expression.cpp index aa5266173..1d9ba299b 100644 --- a/src/duckdb/ub_src_parser_expression.cpp +++ b/src/duckdb/ub_src_parser_expression.cpp @@ -20,6 +20,8 @@ #include "src/parser/expression/lambda_expression.cpp" +#include "src/parser/expression/lambdaref_expression.cpp" + #include "src/parser/expression/operator_expression.cpp" #include "src/parser/expression/parameter_expression.cpp" diff --git a/src/duckdb/ub_src_storage_statistics.cpp b/src/duckdb/ub_src_storage_statistics.cpp index 3cb355a31..044f55741 100644 --- a/src/duckdb/ub_src_storage_statistics.cpp +++ b/src/duckdb/ub_src_storage_statistics.cpp @@ -4,6 +4,8 @@ #include "src/storage/statistics/distinct_statistics.cpp" +#include "src/storage/statistics/array_stats.cpp" + #include "src/storage/statistics/list_stats.cpp" #include "src/storage/statistics/numeric_stats.cpp" diff --git a/src/duckdb/ub_src_storage_table.cpp b/src/duckdb/ub_src_storage_table.cpp index 3e3e9335a..c7f69ecd0 100644 --- a/src/duckdb/ub_src_storage_table.cpp +++ b/src/duckdb/ub_src_storage_table.cpp @@ -8,6 +8,8 @@ #include "src/storage/table/column_segment.cpp" +#include "src/storage/table/array_column_data.cpp" + #include "src/storage/table/list_column_data.cpp" #include "src/storage/table/update_segment.cpp" diff --git a/src/include/sources.mk b/src/include/sources.mk index f8ceb870e..e65445039 100644 --- a/src/include/sources.mk +++ b/src/include/sources.mk @@ -1 +1 @@ -SOURCES=duckdb/ub_src_catalog.o duckdb/ub_src_catalog_catalog_entry.o duckdb/ub_src_catalog_default.o duckdb/ub_src_common_adbc.o duckdb/ub_src_common_adbc_nanoarrow.o duckdb/ub_src_common.o duckdb/ub_src_common_arrow_appender.o duckdb/ub_src_common_arrow.o duckdb/ub_src_common_crypto.o duckdb/ub_src_common_enums.o duckdb/ub_src_common_operator.o duckdb/ub_src_common_progress_bar.o duckdb/ub_src_common_row_operations.o duckdb/ub_src_common_serializer.o duckdb/ub_src_common_sort.o duckdb/ub_src_common_types.o duckdb/ub_src_common_types_column.o duckdb/ub_src_common_types_row.o duckdb/ub_src_common_value_operations.o duckdb/src/common/vector_operations/boolean_operators.o duckdb/src/common/vector_operations/comparison_operators.o duckdb/src/common/vector_operations/generators.o duckdb/src/common/vector_operations/is_distinct_from.o duckdb/src/common/vector_operations/null_operations.o duckdb/src/common/vector_operations/numeric_inplace_operators.o duckdb/src/common/vector_operations/vector_cast.o duckdb/src/common/vector_operations/vector_copy.o duckdb/src/common/vector_operations/vector_hash.o duckdb/src/common/vector_operations/vector_storage.o duckdb/ub_src_core_functions_aggregate_algebraic.o duckdb/ub_src_core_functions_aggregate_distributive.o duckdb/ub_src_core_functions_aggregate_holistic.o duckdb/ub_src_core_functions_aggregate_nested.o duckdb/ub_src_core_functions_aggregate_regression.o duckdb/ub_src_core_functions.o duckdb/ub_src_core_functions_scalar_bit.o duckdb/ub_src_core_functions_scalar_blob.o duckdb/ub_src_core_functions_scalar_date.o duckdb/ub_src_core_functions_scalar_debug.o duckdb/ub_src_core_functions_scalar_enum.o duckdb/ub_src_core_functions_scalar_generic.o duckdb/ub_src_core_functions_scalar_list.o duckdb/ub_src_core_functions_scalar_map.o duckdb/ub_src_core_functions_scalar_math.o duckdb/ub_src_core_functions_scalar_operators.o duckdb/ub_src_core_functions_scalar_random.o duckdb/ub_src_core_functions_scalar_string.o duckdb/ub_src_core_functions_scalar_struct.o duckdb/ub_src_core_functions_scalar_union.o duckdb/ub_src_execution.o duckdb/ub_src_execution_expression_executor.o duckdb/ub_src_execution_index_art.o duckdb/ub_src_execution_index.o duckdb/ub_src_execution_nested_loop_join.o duckdb/ub_src_execution_operator_aggregate.o duckdb/ub_src_execution_operator_csv_scanner.o duckdb/ub_src_execution_operator_csv_scanner_sniffer.o duckdb/ub_src_execution_operator_filter.o duckdb/ub_src_execution_operator_helper.o duckdb/ub_src_execution_operator_join.o duckdb/ub_src_execution_operator_order.o duckdb/ub_src_execution_operator_persistent.o duckdb/ub_src_execution_operator_projection.o duckdb/ub_src_execution_operator_scan.o duckdb/ub_src_execution_operator_schema.o duckdb/ub_src_execution_operator_set.o duckdb/ub_src_execution_physical_plan.o duckdb/ub_src_function_aggregate_distributive.o duckdb/ub_src_function_aggregate.o duckdb/ub_src_function.o duckdb/ub_src_function_cast.o duckdb/ub_src_function_cast_union.o duckdb/ub_src_function_pragma.o duckdb/ub_src_function_scalar_compressed_materialization.o duckdb/ub_src_function_scalar.o duckdb/ub_src_function_scalar_generic.o duckdb/ub_src_function_scalar_list.o duckdb/ub_src_function_scalar_operators.o duckdb/ub_src_function_scalar_sequence.o duckdb/ub_src_function_scalar_string.o duckdb/ub_src_function_scalar_string_regexp.o duckdb/ub_src_function_scalar_struct.o duckdb/ub_src_function_scalar_system.o duckdb/ub_src_function_table_arrow.o duckdb/ub_src_function_table.o duckdb/ub_src_function_table_system.o duckdb/ub_src_function_table_version.o duckdb/ub_src_main.o duckdb/ub_src_main_capi.o duckdb/ub_src_main_capi_cast.o duckdb/ub_src_main_chunk_scan_state.o duckdb/ub_src_main_extension.o duckdb/ub_src_main_relation.o duckdb/ub_src_main_settings.o duckdb/ub_src_optimizer.o duckdb/ub_src_optimizer_compressed_materialization.o duckdb/ub_src_optimizer_join_order.o duckdb/ub_src_optimizer_matcher.o duckdb/ub_src_optimizer_pullup.o duckdb/ub_src_optimizer_pushdown.o duckdb/ub_src_optimizer_rule.o duckdb/ub_src_optimizer_statistics_expression.o duckdb/ub_src_optimizer_statistics_operator.o duckdb/ub_src_parallel.o duckdb/ub_src_parser.o duckdb/ub_src_parser_constraints.o duckdb/ub_src_parser_expression.o duckdb/ub_src_parser_parsed_data.o duckdb/ub_src_parser_query_node.o duckdb/ub_src_parser_statement.o duckdb/ub_src_parser_tableref.o duckdb/ub_src_parser_transform_constraint.o duckdb/ub_src_parser_transform_expression.o duckdb/ub_src_parser_transform_helpers.o duckdb/ub_src_parser_transform_statement.o duckdb/ub_src_parser_transform_tableref.o duckdb/ub_src_planner.o duckdb/ub_src_planner_binder_expression.o duckdb/ub_src_planner_binder_query_node.o duckdb/ub_src_planner_binder_statement.o duckdb/ub_src_planner_binder_tableref.o duckdb/ub_src_planner_expression.o duckdb/ub_src_planner_expression_binder.o duckdb/ub_src_planner_filter.o duckdb/ub_src_planner_operator.o duckdb/ub_src_planner_subquery.o duckdb/ub_src_storage.o duckdb/ub_src_storage_buffer.o duckdb/ub_src_storage_checkpoint.o duckdb/ub_src_storage_compression.o duckdb/ub_src_storage_compression_chimp.o duckdb/ub_src_storage_metadata.o duckdb/ub_src_storage_serialization.o duckdb/ub_src_storage_statistics.o duckdb/ub_src_storage_table.o duckdb/ub_src_transaction.o duckdb/src/verification/copied_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier.o duckdb/src/verification/external_statement_verifier.o duckdb/src/verification/no_operator_caching_verifier.o duckdb/src/verification/parsed_statement_verifier.o duckdb/src/verification/prepared_statement_verifier.o duckdb/src/verification/statement_verifier.o duckdb/src/verification/unoptimized_statement_verifier.o duckdb/third_party/fmt/format.o duckdb/third_party/fsst/fsst_avx512.o duckdb/third_party/fsst/libfsst.o duckdb/third_party/miniz/miniz.o duckdb/third_party/re2/re2/bitstate.o duckdb/third_party/re2/re2/compile.o duckdb/third_party/re2/re2/dfa.o duckdb/third_party/re2/re2/filtered_re2.o duckdb/third_party/re2/re2/mimics_pcre.o duckdb/third_party/re2/re2/nfa.o duckdb/third_party/re2/re2/onepass.o duckdb/third_party/re2/re2/parse.o duckdb/third_party/re2/re2/perl_groups.o duckdb/third_party/re2/re2/prefilter.o duckdb/third_party/re2/re2/prefilter_tree.o duckdb/third_party/re2/re2/prog.o duckdb/third_party/re2/re2/re2.o duckdb/third_party/re2/re2/regexp.o duckdb/third_party/re2/re2/set.o duckdb/third_party/re2/re2/simplify.o duckdb/third_party/re2/re2/stringpiece.o duckdb/third_party/re2/re2/tostring.o duckdb/third_party/re2/re2/unicode_casefold.o duckdb/third_party/re2/re2/unicode_groups.o duckdb/third_party/re2/util/rune.o duckdb/third_party/re2/util/strutil.o duckdb/third_party/hyperloglog/hyperloglog.o duckdb/third_party/hyperloglog/sds.o duckdb/third_party/fastpforlib/bitpacking.o duckdb/third_party/utf8proc/utf8proc.o duckdb/third_party/utf8proc/utf8proc_wrapper.o duckdb/third_party/libpg_query/pg_functions.o duckdb/third_party/libpg_query/postgres_parser.o duckdb/third_party/libpg_query/src_backend_nodes_list.o duckdb/third_party/libpg_query/src_backend_nodes_makefuncs.o duckdb/third_party/libpg_query/src_backend_nodes_value.o duckdb/third_party/libpg_query/src_backend_parser_gram.o duckdb/third_party/libpg_query/src_backend_parser_parser.o duckdb/third_party/libpg_query/src_backend_parser_scan.o duckdb/third_party/libpg_query/src_backend_parser_scansup.o duckdb/third_party/libpg_query/src_common_keywords.o duckdb/third_party/mbedtls/library/asn1parse.o duckdb/third_party/mbedtls/library/base64.o duckdb/third_party/mbedtls/library/bignum.o duckdb/third_party/mbedtls/library/constant_time.o duckdb/third_party/mbedtls/library/md.o duckdb/third_party/mbedtls/library/oid.o duckdb/third_party/mbedtls/library/pem.o duckdb/third_party/mbedtls/library/pk.o duckdb/third_party/mbedtls/library/pk_wrap.o duckdb/third_party/mbedtls/library/pkparse.o duckdb/third_party/mbedtls/library/platform_util.o duckdb/third_party/mbedtls/library/rsa.o duckdb/third_party/mbedtls/library/rsa_alt_helpers.o duckdb/third_party/mbedtls/library/sha1.o duckdb/third_party/mbedtls/library/sha256.o duckdb/third_party/mbedtls/library/sha512.o duckdb/third_party/mbedtls/mbedtls_wrapper.o duckdb/extension/parquet/parquet_extension.o duckdb/extension/parquet/column_writer.o duckdb/extension/parquet/serialize_parquet.o duckdb/extension/parquet/parquet_reader.o duckdb/extension/parquet/parquet_timestamp.o duckdb/extension/parquet/parquet_writer.o duckdb/extension/parquet/column_reader.o duckdb/extension/parquet/parquet_statistics.o duckdb/extension/parquet/parquet_metadata.o duckdb/extension/parquet/zstd_file_system.o duckdb/third_party/parquet/parquet_constants.o duckdb/third_party/parquet/parquet_types.o duckdb/third_party/thrift/thrift/protocol/TProtocol.o duckdb/third_party/thrift/thrift/transport/TTransportException.o duckdb/third_party/thrift/thrift/transport/TBufferTransports.o duckdb/third_party/snappy/snappy.o duckdb/third_party/snappy/snappy-sinksource.o duckdb/third_party/zstd/decompress/zstd_ddict.o duckdb/third_party/zstd/decompress/huf_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress_block.o duckdb/third_party/zstd/common/entropy_common.o duckdb/third_party/zstd/common/fse_decompress.o duckdb/third_party/zstd/common/zstd_common.o duckdb/third_party/zstd/common/error_private.o duckdb/third_party/zstd/common/xxhash.o duckdb/third_party/zstd/compress/fse_compress.o duckdb/third_party/zstd/compress/hist.o duckdb/third_party/zstd/compress/huf_compress.o duckdb/third_party/zstd/compress/zstd_compress.o duckdb/third_party/zstd/compress/zstd_compress_literals.o duckdb/third_party/zstd/compress/zstd_compress_sequences.o duckdb/third_party/zstd/compress/zstd_compress_superblock.o duckdb/third_party/zstd/compress/zstd_double_fast.o duckdb/third_party/zstd/compress/zstd_fast.o duckdb/third_party/zstd/compress/zstd_lazy.o duckdb/third_party/zstd/compress/zstd_ldm.o duckdb/third_party/zstd/compress/zstd_opt.o +SOURCES=duckdb/ub_src_catalog.o duckdb/ub_src_catalog_catalog_entry.o duckdb/ub_src_catalog_default.o duckdb/ub_src_common_adbc.o duckdb/ub_src_common_adbc_nanoarrow.o duckdb/ub_src_common.o duckdb/ub_src_common_arrow_appender.o duckdb/ub_src_common_arrow.o duckdb/ub_src_common_crypto.o duckdb/ub_src_common_enums.o duckdb/ub_src_common_operator.o duckdb/ub_src_common_progress_bar.o duckdb/ub_src_common_row_operations.o duckdb/ub_src_common_serializer.o duckdb/ub_src_common_sort.o duckdb/ub_src_common_types.o duckdb/ub_src_common_types_column.o duckdb/ub_src_common_types_row.o duckdb/ub_src_common_value_operations.o duckdb/src/common/vector_operations/boolean_operators.o duckdb/src/common/vector_operations/comparison_operators.o duckdb/src/common/vector_operations/generators.o duckdb/src/common/vector_operations/is_distinct_from.o duckdb/src/common/vector_operations/null_operations.o duckdb/src/common/vector_operations/numeric_inplace_operators.o duckdb/src/common/vector_operations/vector_cast.o duckdb/src/common/vector_operations/vector_copy.o duckdb/src/common/vector_operations/vector_hash.o duckdb/src/common/vector_operations/vector_storage.o duckdb/ub_src_core_functions_aggregate_algebraic.o duckdb/ub_src_core_functions_aggregate_distributive.o duckdb/ub_src_core_functions_aggregate_holistic.o duckdb/ub_src_core_functions_aggregate_nested.o duckdb/ub_src_core_functions_aggregate_regression.o duckdb/ub_src_core_functions.o duckdb/ub_src_core_functions_scalar_array.o duckdb/ub_src_core_functions_scalar_bit.o duckdb/ub_src_core_functions_scalar_blob.o duckdb/ub_src_core_functions_scalar_date.o duckdb/ub_src_core_functions_scalar_debug.o duckdb/ub_src_core_functions_scalar_enum.o duckdb/ub_src_core_functions_scalar_generic.o duckdb/ub_src_core_functions_scalar_list.o duckdb/ub_src_core_functions_scalar_map.o duckdb/ub_src_core_functions_scalar_math.o duckdb/ub_src_core_functions_scalar_operators.o duckdb/ub_src_core_functions_scalar_random.o duckdb/ub_src_core_functions_scalar_string.o duckdb/ub_src_core_functions_scalar_struct.o duckdb/ub_src_core_functions_scalar_union.o duckdb/ub_src_execution.o duckdb/ub_src_execution_expression_executor.o duckdb/ub_src_execution_index_art.o duckdb/ub_src_execution_index.o duckdb/ub_src_execution_nested_loop_join.o duckdb/ub_src_execution_operator_aggregate.o duckdb/ub_src_execution_operator_csv_scanner.o duckdb/ub_src_execution_operator_csv_scanner_sniffer.o duckdb/ub_src_execution_operator_filter.o duckdb/ub_src_execution_operator_helper.o duckdb/ub_src_execution_operator_join.o duckdb/ub_src_execution_operator_order.o duckdb/ub_src_execution_operator_persistent.o duckdb/ub_src_execution_operator_projection.o duckdb/ub_src_execution_operator_scan.o duckdb/ub_src_execution_operator_schema.o duckdb/ub_src_execution_operator_set.o duckdb/ub_src_execution_physical_plan.o duckdb/ub_src_function_aggregate_distributive.o duckdb/ub_src_function_aggregate.o duckdb/ub_src_function.o duckdb/ub_src_function_cast.o duckdb/ub_src_function_cast_union.o duckdb/ub_src_function_pragma.o duckdb/ub_src_function_scalar_compressed_materialization.o duckdb/ub_src_function_scalar.o duckdb/ub_src_function_scalar_generic.o duckdb/ub_src_function_scalar_list.o duckdb/ub_src_function_scalar_operators.o duckdb/ub_src_function_scalar_sequence.o duckdb/ub_src_function_scalar_string.o duckdb/ub_src_function_scalar_string_regexp.o duckdb/ub_src_function_scalar_struct.o duckdb/ub_src_function_scalar_system.o duckdb/ub_src_function_table_arrow.o duckdb/ub_src_function_table.o duckdb/ub_src_function_table_system.o duckdb/ub_src_function_table_version.o duckdb/ub_src_main.o duckdb/ub_src_main_capi.o duckdb/ub_src_main_capi_cast.o duckdb/ub_src_main_chunk_scan_state.o duckdb/ub_src_main_extension.o duckdb/ub_src_main_relation.o duckdb/ub_src_main_settings.o duckdb/ub_src_optimizer.o duckdb/ub_src_optimizer_compressed_materialization.o duckdb/ub_src_optimizer_join_order.o duckdb/ub_src_optimizer_matcher.o duckdb/ub_src_optimizer_pullup.o duckdb/ub_src_optimizer_pushdown.o duckdb/ub_src_optimizer_rule.o duckdb/ub_src_optimizer_statistics_expression.o duckdb/ub_src_optimizer_statistics_operator.o duckdb/ub_src_parallel.o duckdb/ub_src_parser.o duckdb/ub_src_parser_constraints.o duckdb/ub_src_parser_expression.o duckdb/ub_src_parser_parsed_data.o duckdb/ub_src_parser_query_node.o duckdb/ub_src_parser_statement.o duckdb/ub_src_parser_tableref.o duckdb/ub_src_parser_transform_constraint.o duckdb/ub_src_parser_transform_expression.o duckdb/ub_src_parser_transform_helpers.o duckdb/ub_src_parser_transform_statement.o duckdb/ub_src_parser_transform_tableref.o duckdb/ub_src_planner.o duckdb/ub_src_planner_binder_expression.o duckdb/ub_src_planner_binder_query_node.o duckdb/ub_src_planner_binder_statement.o duckdb/ub_src_planner_binder_tableref.o duckdb/ub_src_planner_expression.o duckdb/ub_src_planner_expression_binder.o duckdb/ub_src_planner_filter.o duckdb/ub_src_planner_operator.o duckdb/ub_src_planner_subquery.o duckdb/ub_src_storage.o duckdb/ub_src_storage_buffer.o duckdb/ub_src_storage_checkpoint.o duckdb/ub_src_storage_compression.o duckdb/ub_src_storage_compression_chimp.o duckdb/ub_src_storage_metadata.o duckdb/ub_src_storage_serialization.o duckdb/ub_src_storage_statistics.o duckdb/ub_src_storage_table.o duckdb/ub_src_transaction.o duckdb/src/verification/copied_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier.o duckdb/src/verification/external_statement_verifier.o duckdb/src/verification/no_operator_caching_verifier.o duckdb/src/verification/parsed_statement_verifier.o duckdb/src/verification/prepared_statement_verifier.o duckdb/src/verification/statement_verifier.o duckdb/src/verification/unoptimized_statement_verifier.o duckdb/third_party/fmt/format.o duckdb/third_party/fsst/fsst_avx512.o duckdb/third_party/fsst/libfsst.o duckdb/third_party/miniz/miniz.o duckdb/third_party/re2/re2/bitstate.o duckdb/third_party/re2/re2/compile.o duckdb/third_party/re2/re2/dfa.o duckdb/third_party/re2/re2/filtered_re2.o duckdb/third_party/re2/re2/mimics_pcre.o duckdb/third_party/re2/re2/nfa.o duckdb/third_party/re2/re2/onepass.o duckdb/third_party/re2/re2/parse.o duckdb/third_party/re2/re2/perl_groups.o duckdb/third_party/re2/re2/prefilter.o duckdb/third_party/re2/re2/prefilter_tree.o duckdb/third_party/re2/re2/prog.o duckdb/third_party/re2/re2/re2.o duckdb/third_party/re2/re2/regexp.o duckdb/third_party/re2/re2/set.o duckdb/third_party/re2/re2/simplify.o duckdb/third_party/re2/re2/stringpiece.o duckdb/third_party/re2/re2/tostring.o duckdb/third_party/re2/re2/unicode_casefold.o duckdb/third_party/re2/re2/unicode_groups.o duckdb/third_party/re2/util/rune.o duckdb/third_party/re2/util/strutil.o duckdb/third_party/hyperloglog/hyperloglog.o duckdb/third_party/hyperloglog/sds.o duckdb/third_party/fastpforlib/bitpacking.o duckdb/third_party/utf8proc/utf8proc.o duckdb/third_party/utf8proc/utf8proc_wrapper.o duckdb/third_party/libpg_query/pg_functions.o duckdb/third_party/libpg_query/postgres_parser.o duckdb/third_party/libpg_query/src_backend_nodes_list.o duckdb/third_party/libpg_query/src_backend_nodes_makefuncs.o duckdb/third_party/libpg_query/src_backend_nodes_value.o duckdb/third_party/libpg_query/src_backend_parser_gram.o duckdb/third_party/libpg_query/src_backend_parser_parser.o duckdb/third_party/libpg_query/src_backend_parser_scan.o duckdb/third_party/libpg_query/src_backend_parser_scansup.o duckdb/third_party/libpg_query/src_common_keywords.o duckdb/third_party/mbedtls/library/asn1parse.o duckdb/third_party/mbedtls/library/base64.o duckdb/third_party/mbedtls/library/bignum.o duckdb/third_party/mbedtls/library/constant_time.o duckdb/third_party/mbedtls/library/md.o duckdb/third_party/mbedtls/library/oid.o duckdb/third_party/mbedtls/library/pem.o duckdb/third_party/mbedtls/library/pk.o duckdb/third_party/mbedtls/library/pk_wrap.o duckdb/third_party/mbedtls/library/pkparse.o duckdb/third_party/mbedtls/library/platform_util.o duckdb/third_party/mbedtls/library/rsa.o duckdb/third_party/mbedtls/library/rsa_alt_helpers.o duckdb/third_party/mbedtls/library/sha1.o duckdb/third_party/mbedtls/library/sha256.o duckdb/third_party/mbedtls/library/sha512.o duckdb/third_party/mbedtls/mbedtls_wrapper.o duckdb/extension/parquet/parquet_extension.o duckdb/extension/parquet/column_writer.o duckdb/extension/parquet/serialize_parquet.o duckdb/extension/parquet/parquet_reader.o duckdb/extension/parquet/parquet_timestamp.o duckdb/extension/parquet/parquet_writer.o duckdb/extension/parquet/column_reader.o duckdb/extension/parquet/parquet_statistics.o duckdb/extension/parquet/parquet_metadata.o duckdb/extension/parquet/zstd_file_system.o duckdb/third_party/parquet/parquet_constants.o duckdb/third_party/parquet/parquet_types.o duckdb/third_party/thrift/thrift/protocol/TProtocol.o duckdb/third_party/thrift/thrift/transport/TTransportException.o duckdb/third_party/thrift/thrift/transport/TBufferTransports.o duckdb/third_party/snappy/snappy.o duckdb/third_party/snappy/snappy-sinksource.o duckdb/third_party/zstd/decompress/zstd_ddict.o duckdb/third_party/zstd/decompress/huf_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress_block.o duckdb/third_party/zstd/common/entropy_common.o duckdb/third_party/zstd/common/fse_decompress.o duckdb/third_party/zstd/common/zstd_common.o duckdb/third_party/zstd/common/error_private.o duckdb/third_party/zstd/common/xxhash.o duckdb/third_party/zstd/compress/fse_compress.o duckdb/third_party/zstd/compress/hist.o duckdb/third_party/zstd/compress/huf_compress.o duckdb/third_party/zstd/compress/zstd_compress.o duckdb/third_party/zstd/compress/zstd_compress_literals.o duckdb/third_party/zstd/compress/zstd_compress_sequences.o duckdb/third_party/zstd/compress/zstd_compress_superblock.o duckdb/third_party/zstd/compress/zstd_double_fast.o duckdb/third_party/zstd/compress/zstd_fast.o duckdb/third_party/zstd/compress/zstd_lazy.o duckdb/third_party/zstd/compress/zstd_ldm.o duckdb/third_party/zstd/compress/zstd_opt.o